Merge branch 'master' into renew_syn_on_merge
This commit is contained in:
commit
edef67ffdc
|
@ -27,12 +27,14 @@ import org.elasticsearch.common.component.AbstractComponent;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.analysis.NamedAnalyzer;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
import static java.util.Collections.unmodifiableSet;
|
||||
|
@ -213,14 +215,15 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
|
|||
* Checks the mappings for compatibility with the current version
|
||||
*/
|
||||
private void checkMappingsCompatibility(IndexMetaData indexMetaData) {
|
||||
Index index = new Index(indexMetaData.getIndex());
|
||||
Settings settings = indexMetaData.getSettings();
|
||||
try {
|
||||
SimilarityService similarityService = new SimilarityService(index, settings);
|
||||
// We cannot instantiate real analysis server at this point because the node might not have
|
||||
// been started yet. However, we don't really need real analyzers at this stage - so we can fake it
|
||||
try (AnalysisService analysisService = new FakeAnalysisService(index, settings)) {
|
||||
try (MapperService mapperService = new MapperService(index, settings, analysisService, similarityService, scriptService)) {
|
||||
IndexSettings indexSettings = new IndexSettings(indexMetaData, this.settings, Collections.EMPTY_LIST);
|
||||
SimilarityService similarityService = new SimilarityService(indexSettings, Collections.EMPTY_MAP);
|
||||
|
||||
|
||||
try (AnalysisService analysisService = new FakeAnalysisService(indexSettings)) {
|
||||
try (MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, scriptService)) {
|
||||
for (ObjectCursor<MappingMetaData> cursor : indexMetaData.getMappings().values()) {
|
||||
MappingMetaData mappingMetaData = cursor.value;
|
||||
mapperService.merge(mappingMetaData.type(), mappingMetaData.source(), false, false);
|
||||
|
@ -253,8 +256,8 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
|
|||
}
|
||||
};
|
||||
|
||||
public FakeAnalysisService(Index index, Settings indexSettings) {
|
||||
super(index, indexSettings);
|
||||
public FakeAnalysisService(IndexSettings indexSettings) {
|
||||
super(indexSettings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -511,7 +511,9 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
|||
continue;
|
||||
}
|
||||
RoutingNode target = routingNodes.node(currentNode.getNodeId());
|
||||
Decision decision = allocation.deciders().canAllocate(shard, target, allocation);
|
||||
Decision allocationDecision = allocation.deciders().canAllocate(shard, target, allocation);
|
||||
Decision rebalanceDecision = allocation.deciders().canRebalance(shard, allocation);
|
||||
Decision decision = new Decision.Multi().add(allocationDecision).add(rebalanceDecision);
|
||||
if (decision.type() == Type.YES) { // TODO maybe we can respect throttling here too?
|
||||
sourceNode.removeShard(shard);
|
||||
ShardRouting targetRelocatingShard = routingNodes.relocate(shard, target.nodeId(), allocation.clusterInfo().getShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.elasticsearch.common.io.FileSystemUtils;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.store.FsDirectoryService;
|
||||
import org.elasticsearch.monitor.fs.FsInfo;
|
||||
|
@ -309,7 +308,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @param shardId the id of the shard to delete to delete
|
||||
* @throws IOException if an IOException occurs
|
||||
*/
|
||||
public void deleteShardDirectorySafe(ShardId shardId, @IndexSettings Settings indexSettings) throws IOException {
|
||||
public void deleteShardDirectorySafe(ShardId shardId, Settings indexSettings) throws IOException {
|
||||
// This is to ensure someone doesn't use Settings.EMPTY
|
||||
assert indexSettings != Settings.EMPTY;
|
||||
final Path[] paths = availableShardPaths(shardId);
|
||||
|
@ -326,7 +325,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
*
|
||||
* @throws LockObtainFailedException if any of the locks could not be acquired
|
||||
*/
|
||||
public static void acquireFSLockForPaths(@IndexSettings Settings indexSettings, Path... shardPaths) throws IOException {
|
||||
public static void acquireFSLockForPaths(Settings indexSettings, Path... shardPaths) throws IOException {
|
||||
Lock[] locks = new Lock[shardPaths.length];
|
||||
Directory[] dirs = new Directory[shardPaths.length];
|
||||
try {
|
||||
|
@ -360,7 +359,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @throws IOException if an IOException occurs
|
||||
* @throws ElasticsearchException if the write.lock is not acquirable
|
||||
*/
|
||||
public void deleteShardDirectoryUnderLock(ShardLock lock, @IndexSettings Settings indexSettings) throws IOException {
|
||||
public void deleteShardDirectoryUnderLock(ShardLock lock, Settings indexSettings) throws IOException {
|
||||
assert indexSettings != Settings.EMPTY;
|
||||
final ShardId shardId = lock.getShardId();
|
||||
assert isShardLocked(shardId) : "shard " + shardId + " is not locked";
|
||||
|
@ -398,7 +397,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @param indexSettings settings for the index being deleted
|
||||
* @throws IOException if any of the shards data directories can't be locked or deleted
|
||||
*/
|
||||
public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, @IndexSettings Settings indexSettings) throws IOException {
|
||||
public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, Settings indexSettings) throws IOException {
|
||||
// This is to ensure someone doesn't use Settings.EMPTY
|
||||
assert indexSettings != Settings.EMPTY;
|
||||
final List<ShardLock> locks = lockAllForIndex(index, indexSettings, lockTimeoutMS);
|
||||
|
@ -416,7 +415,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @param index the index to delete
|
||||
* @param indexSettings settings for the index being deleted
|
||||
*/
|
||||
public void deleteIndexDirectoryUnderLock(Index index, @IndexSettings Settings indexSettings) throws IOException {
|
||||
public void deleteIndexDirectoryUnderLock(Index index, Settings indexSettings) throws IOException {
|
||||
// This is to ensure someone doesn't use Settings.EMPTY
|
||||
assert indexSettings != Settings.EMPTY;
|
||||
final Path[] indexPaths = indexPaths(index);
|
||||
|
@ -439,7 +438,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @return the {@link ShardLock} instances for this index.
|
||||
* @throws IOException if an IOException occurs.
|
||||
*/
|
||||
public List<ShardLock> lockAllForIndex(Index index, @IndexSettings Settings settings, long lockTimeoutMS) throws IOException {
|
||||
public List<ShardLock> lockAllForIndex(Index index, Settings settings, long lockTimeoutMS) throws IOException {
|
||||
final Integer numShards = settings.getAsInt(IndexMetaData.SETTING_NUMBER_OF_SHARDS, null);
|
||||
if (numShards == null || numShards <= 0) {
|
||||
throw new IllegalArgumentException("settings must contain a non-null > 0 number of shards");
|
||||
|
@ -785,7 +784,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @param indexSettings settings for an index
|
||||
* @return true if the index has a custom data path
|
||||
*/
|
||||
public static boolean hasCustomDataPath(@IndexSettings Settings indexSettings) {
|
||||
public static boolean hasCustomDataPath(Settings indexSettings) {
|
||||
return indexSettings.get(IndexMetaData.SETTING_DATA_PATH) != null;
|
||||
}
|
||||
|
||||
|
@ -796,7 +795,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
*
|
||||
* @param indexSettings settings for the index
|
||||
*/
|
||||
private Path resolveCustomLocation(@IndexSettings Settings indexSettings) {
|
||||
private Path resolveCustomLocation(Settings indexSettings) {
|
||||
assert indexSettings != Settings.EMPTY;
|
||||
String customDataDir = indexSettings.get(IndexMetaData.SETTING_DATA_PATH);
|
||||
if (customDataDir != null) {
|
||||
|
@ -820,7 +819,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @param indexSettings settings for the index
|
||||
* @param indexName index to resolve the path for
|
||||
*/
|
||||
private Path resolveCustomLocation(@IndexSettings Settings indexSettings, final String indexName) {
|
||||
private Path resolveCustomLocation(Settings indexSettings, final String indexName) {
|
||||
return resolveCustomLocation(indexSettings).resolve(indexName);
|
||||
}
|
||||
|
||||
|
@ -832,7 +831,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
|
|||
* @param indexSettings settings for the index
|
||||
* @param shardId shard to resolve the path to
|
||||
*/
|
||||
public Path resolveCustomLocation(@IndexSettings Settings indexSettings, final ShardId shardId) {
|
||||
public Path resolveCustomLocation(Settings indexSettings, final ShardId shardId) {
|
||||
return resolveCustomLocation(indexSettings, shardId.index().name()).resolve(Integer.toString(shardId.id()));
|
||||
}
|
||||
|
||||
|
|
|
@ -30,15 +30,8 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
|||
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
* The primary shard allocator allocates primary shard that were not created as
|
||||
|
@ -77,8 +70,9 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
|
|||
}
|
||||
|
||||
IndexMetaData indexMetaData = metaData.index(shard.getIndex());
|
||||
Settings indexSettings = Settings.builder().put(settings).put(indexMetaData.getSettings()).build();
|
||||
|
||||
NodesAndVersions nodesAndVersions = buildNodesAndVersions(shard, recoverOnAnyNode(indexMetaData.getSettings()), allocation.getIgnoreNodes(shard.shardId()), shardState);
|
||||
NodesAndVersions nodesAndVersions = buildNodesAndVersions(shard, recoverOnAnyNode(indexSettings), allocation.getIgnoreNodes(shard.shardId()), shardState);
|
||||
logger.debug("[{}][{}] found {} allocations of {}, highest version: [{}]", shard.index(), shard.id(), nodesAndVersions.allocationsFound, shard, nodesAndVersions.highestVersion);
|
||||
|
||||
if (isEnoughAllocationsFound(shard, indexMetaData, nodesAndVersions) == false) {
|
||||
|
@ -263,7 +257,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
|
|||
* Return {@code true} if the index is configured to allow shards to be
|
||||
* recovered on any node
|
||||
*/
|
||||
private boolean recoverOnAnyNode(@IndexSettings Settings idxSettings) {
|
||||
private boolean recoverOnAnyNode(Settings idxSettings) {
|
||||
return IndexMetaData.isOnSharedFilesystem(idxSettings) &&
|
||||
idxSettings.getAsBoolean(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, false);
|
||||
}
|
||||
|
|
|
@ -39,6 +39,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.ShardPath;
|
||||
import org.elasticsearch.index.shard.ShardStateMetaData;
|
||||
|
@ -48,6 +50,7 @@ import org.elasticsearch.transport.TransportService;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicReferenceArray;
|
||||
|
||||
|
@ -128,7 +131,8 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
|
|||
if (metaData != null) {
|
||||
ShardPath shardPath = null;
|
||||
try {
|
||||
shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, metaData.getSettings());
|
||||
IndexSettings indexSettings = new IndexSettings(metaData, settings, Collections.EMPTY_LIST);
|
||||
shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, indexSettings);
|
||||
if (shardPath == null) {
|
||||
throw new IllegalStateException(shardId + " no shard path found");
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.common.logging.DeprecationLogger;
|
|||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -32,32 +31,23 @@ public abstract class AbstractIndexComponent implements IndexComponent {
|
|||
|
||||
protected final ESLogger logger;
|
||||
protected final DeprecationLogger deprecationLogger;
|
||||
protected final Index index;
|
||||
protected final Settings indexSettings;
|
||||
protected final IndexSettings indexSettings;
|
||||
|
||||
/**
|
||||
* Constructs a new index component, with the index name and its settings.
|
||||
*
|
||||
* @param index The index name
|
||||
* @param indexSettings The index settings
|
||||
*/
|
||||
protected AbstractIndexComponent(Index index, @IndexSettings Settings indexSettings) {
|
||||
this.index = index;
|
||||
this.indexSettings = indexSettings;
|
||||
this.logger = Loggers.getLogger(getClass(), indexSettings, index);
|
||||
protected AbstractIndexComponent(IndexSettings indexSettings) {
|
||||
this.logger = Loggers.getLogger(getClass(), indexSettings.getSettings(), indexSettings.getIndex());
|
||||
this.deprecationLogger = new DeprecationLogger(logger);
|
||||
this.indexSettings = indexSettings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Index index() {
|
||||
return this.index;
|
||||
return indexSettings.getIndex();
|
||||
}
|
||||
|
||||
public Settings indexSettings() {
|
||||
public IndexSettings getIndexSettings() {
|
||||
return indexSettings;
|
||||
}
|
||||
|
||||
public String nodeName() {
|
||||
return indexSettings.get("name", "");
|
||||
}
|
||||
}
|
|
@ -24,7 +24,6 @@ import org.elasticsearch.common.Nullable;
|
|||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.index.shard.IndexEventListener;
|
||||
|
@ -43,14 +42,14 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
private final List<IndexEventListener> listeners;
|
||||
private final ESLogger logger;
|
||||
|
||||
CompositeIndexEventListener(String index, Settings indexSettings, Collection<IndexEventListener> listeners) {
|
||||
CompositeIndexEventListener(IndexSettings indexSettings, Collection<IndexEventListener> listeners) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
if (listener == null) {
|
||||
throw new IllegalArgumentException("listeners must be non-null");
|
||||
}
|
||||
}
|
||||
this.listeners = Collections.unmodifiableList(new ArrayList<>(listeners));
|
||||
this.logger = Loggers.getLogger(getClass(), indexSettings, index);
|
||||
this.logger = Loggers.getLogger(getClass(), indexSettings.getSettings(), indexSettings.getIndex());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -139,7 +138,7 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void beforeIndexCreated(Index index, @IndexSettings Settings indexSettings) {
|
||||
public void beforeIndexCreated(Index index, Settings indexSettings) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
try {
|
||||
listener.beforeIndexCreated(index, indexSettings);
|
||||
|
@ -199,7 +198,7 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void afterIndexDeleted(Index index, @IndexSettings Settings indexSettings) {
|
||||
public void afterIndexDeleted(Index index, Settings indexSettings) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
try {
|
||||
listener.afterIndexDeleted(index, indexSettings);
|
||||
|
@ -211,7 +210,7 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void afterIndexClosed(Index index, @IndexSettings Settings indexSettings) {
|
||||
public void afterIndexClosed(Index index, Settings indexSettings) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
try {
|
||||
listener.afterIndexClosed(index, indexSettings);
|
||||
|
@ -224,7 +223,7 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
|
||||
@Override
|
||||
public void beforeIndexShardDeleted(ShardId shardId,
|
||||
@IndexSettings Settings indexSettings) {
|
||||
Settings indexSettings) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
try {
|
||||
listener.beforeIndexShardDeleted(shardId, indexSettings);
|
||||
|
@ -237,7 +236,7 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
|
||||
@Override
|
||||
public void afterIndexShardDeleted(ShardId shardId,
|
||||
@IndexSettings Settings indexSettings) {
|
||||
Settings indexSettings) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
try {
|
||||
listener.afterIndexShardDeleted(shardId, indexSettings);
|
||||
|
@ -249,7 +248,7 @@ final class CompositeIndexEventListener implements IndexEventListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void beforeIndexAddedToCluster(Index index, @IndexSettings Settings indexSettings) {
|
||||
public void beforeIndexAddedToCluster(Index index, Settings indexSettings) {
|
||||
for (IndexEventListener listener : listeners) {
|
||||
try {
|
||||
listener.beforeIndexAddedToCluster(index, indexSettings);
|
||||
|
|
|
@ -31,32 +31,66 @@ import org.elasticsearch.index.shard.IndexEventListener;
|
|||
import org.elasticsearch.index.shard.IndexSearcherWrapper;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class IndexModule extends AbstractModule {
|
||||
|
||||
private final IndexMetaData indexMetaData;
|
||||
private final Settings settings;
|
||||
private final IndexSettings indexSettings;
|
||||
// pkg private so tests can mock
|
||||
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class;
|
||||
Class<? extends IndexSearcherWrapper> indexSearcherWrapper = null;
|
||||
private final Set<Consumer<Settings>> settingsConsumers = new HashSet<>();
|
||||
private final Set<IndexEventListener> indexEventListeners = new HashSet<>();
|
||||
private IndexEventListener listener;
|
||||
|
||||
|
||||
public IndexModule(Settings settings, IndexMetaData indexMetaData) {
|
||||
this.indexMetaData = indexMetaData;
|
||||
this.settings = settings;
|
||||
public IndexModule(IndexSettings indexSettings) {
|
||||
this.indexSettings = indexSettings;
|
||||
}
|
||||
|
||||
public Settings getIndexSettings() {
|
||||
return settings;
|
||||
/**
|
||||
* Adds a settings consumer for this index
|
||||
*/
|
||||
public void addIndexSettingsListener(Consumer<Settings> listener) {
|
||||
if (listener == null) {
|
||||
throw new IllegalArgumentException("listener must not be null");
|
||||
}
|
||||
|
||||
if (settingsConsumers.contains(listener)) {
|
||||
throw new IllegalStateException("listener already registered");
|
||||
}
|
||||
settingsConsumers.add(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the index {@link Settings} for this index
|
||||
*/
|
||||
public Settings getSettings() {
|
||||
return indexSettings.getSettings();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the index this module is associated with
|
||||
*/
|
||||
public Index getIndex() {
|
||||
return indexSettings.getIndex();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds an {@link IndexEventListener} for this index. All listeners added here
|
||||
* are maintained for the entire index lifecycle on this node. Once an index is closed or deleted these
|
||||
* listeners go out of scope.
|
||||
* <p>
|
||||
* Note: an index might be created on a node multiple times. For instance if the last shard from an index is
|
||||
* relocated to another node the internal representation will be destroyed which includes the registered listeners.
|
||||
* Once the node holds at least one shard of an index all modules are reloaded and listeners are registered again.
|
||||
* Listeners can't be unregistered they will stay alive for the entire time the index is allocated on a node.
|
||||
* </p>
|
||||
*/
|
||||
public void addIndexEventListener(IndexEventListener listener) {
|
||||
if (this.listener != null) {
|
||||
throw new IllegalStateException("can't add listener after listeners are frozen");
|
||||
|
@ -74,7 +108,7 @@ public class IndexModule extends AbstractModule {
|
|||
public IndexEventListener freeze() {
|
||||
// TODO somehow we need to make this pkg private...
|
||||
if (listener == null) {
|
||||
listener = new CompositeIndexEventListener(indexMetaData.getIndex(), settings, indexEventListeners);
|
||||
listener = new CompositeIndexEventListener(indexSettings, indexEventListeners);
|
||||
}
|
||||
return listener;
|
||||
}
|
||||
|
@ -88,10 +122,11 @@ public class IndexModule extends AbstractModule {
|
|||
bind(IndexSearcherWrapper.class).to(indexSearcherWrapper).asEagerSingleton();
|
||||
}
|
||||
bind(IndexEventListener.class).toInstance(freeze());
|
||||
bind(IndexMetaData.class).toInstance(indexMetaData);
|
||||
bind(IndexService.class).asEagerSingleton();
|
||||
bind(IndexServicesProvider.class).asEagerSingleton();
|
||||
bind(MapperService.class).asEagerSingleton();
|
||||
bind(IndexFieldDataService.class).asEagerSingleton();
|
||||
bind(IndexSettings.class).toInstance(new IndexSettings(indexSettings.getIndexMetaData(), indexSettings.getNodeSettings(), settingsConsumers));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,39 +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;
|
||||
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class IndexNameModule extends AbstractModule {
|
||||
|
||||
private final Index index;
|
||||
|
||||
public IndexNameModule(Index index) {
|
||||
this.index = index;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
bind(Index.class).toInstance(index);
|
||||
}
|
||||
}
|
|
@ -24,7 +24,6 @@ import org.apache.lucene.search.BooleanQuery;
|
|||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.cluster.metadata.AliasMetaData;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
|
@ -46,7 +45,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
|
|||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.IndexQueryParserService;
|
||||
import org.elasticsearch.index.query.ParsedQuery;
|
||||
import org.elasticsearch.index.settings.IndexSettingsService;
|
||||
import org.elasticsearch.index.shard.*;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.index.store.IndexStore;
|
||||
|
@ -73,7 +71,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
private final AnalysisService analysisService;
|
||||
private final IndexFieldDataService indexFieldData;
|
||||
private final BitsetFilterCache bitsetFilterCache;
|
||||
private final IndexSettingsService settingsService;
|
||||
private final NodeEnvironment nodeEnv;
|
||||
private final IndicesService indicesServices;
|
||||
private final IndexServicesProvider indexServicesProvider;
|
||||
|
@ -81,30 +78,27 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
private volatile Map<Integer, IndexShard> shards = emptyMap();
|
||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
private final AtomicBoolean deleted = new AtomicBoolean(false);
|
||||
private volatile IndexMetaData indexMetaData;
|
||||
private final IndexSettings indexSettings;
|
||||
|
||||
@Inject
|
||||
public IndexService(Index index, IndexMetaData indexMetaData, NodeEnvironment nodeEnv,
|
||||
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
|
||||
AnalysisService analysisService,
|
||||
IndexSettingsService settingsService,
|
||||
IndexFieldDataService indexFieldData,
|
||||
BitsetFilterCache bitSetFilterCache,
|
||||
IndicesService indicesServices,
|
||||
IndexServicesProvider indexServicesProvider,
|
||||
IndexStore indexStore,
|
||||
IndexEventListener eventListener) {
|
||||
super(index, settingsService.indexSettings());
|
||||
assert indexMetaData != null;
|
||||
super(indexSettings);
|
||||
this.indexSettings = indexSettings;
|
||||
this.analysisService = analysisService;
|
||||
this.indexFieldData = indexFieldData;
|
||||
this.settingsService = settingsService;
|
||||
this.bitsetFilterCache = bitSetFilterCache;
|
||||
this.indicesServices = indicesServices;
|
||||
this.eventListener = eventListener;
|
||||
this.nodeEnv = nodeEnv;
|
||||
this.indexServicesProvider = indexServicesProvider;
|
||||
this.indexStore = indexStore;
|
||||
this.indexMetaData = indexMetaData;
|
||||
indexFieldData.setListener(new FieldDataCacheListener(this));
|
||||
bitSetFilterCache.setListener(new BitsetCacheListener(this));
|
||||
}
|
||||
|
@ -140,18 +134,12 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
public IndexShard getShard(int shardId) {
|
||||
IndexShard indexShard = getShardOrNull(shardId);
|
||||
if (indexShard == null) {
|
||||
throw new ShardNotFoundException(new ShardId(index, shardId));
|
||||
throw new ShardNotFoundException(new ShardId(index(), shardId));
|
||||
}
|
||||
return indexShard;
|
||||
}
|
||||
|
||||
public Set<Integer> shardIds() {
|
||||
return shards.keySet();
|
||||
}
|
||||
|
||||
public IndexSettingsService settingsService() {
|
||||
return this.settingsService;
|
||||
}
|
||||
public Set<Integer> shardIds() { return shards.keySet(); }
|
||||
|
||||
public IndexCache cache() {
|
||||
return indexServicesProvider.getIndexCache();
|
||||
|
@ -197,7 +185,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
|
||||
|
||||
public String indexUUID() {
|
||||
return indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
|
||||
return indexSettings.getUUID();
|
||||
}
|
||||
|
||||
// NOTE: O(numShards) cost, but numShards should be smallish?
|
||||
|
@ -223,10 +211,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
* keep it synced.
|
||||
*/
|
||||
if (closed.get()) {
|
||||
throw new IllegalStateException("Can't create shard [" + index.name() + "][" + sShardId + "], closed");
|
||||
throw new IllegalStateException("Can't create shard [" + index().name() + "][" + sShardId + "], closed");
|
||||
}
|
||||
final Settings indexSettings = settingsService.getSettings();
|
||||
final ShardId shardId = new ShardId(index, sShardId);
|
||||
final Settings indexSettings = this.indexSettings.getSettings();
|
||||
final ShardId shardId = new ShardId(index(), sShardId);
|
||||
boolean success = false;
|
||||
Store store = null;
|
||||
IndexShard indexShard = null;
|
||||
|
@ -235,12 +223,12 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
eventListener.beforeIndexShardCreated(shardId, indexSettings);
|
||||
ShardPath path;
|
||||
try {
|
||||
path = ShardPath.loadShardPath(logger, nodeEnv, shardId, indexSettings);
|
||||
path = ShardPath.loadShardPath(logger, nodeEnv, shardId, this.indexSettings);
|
||||
} catch (IllegalStateException ex) {
|
||||
logger.warn("{} failed to load shard path, trying to remove leftover", shardId);
|
||||
try {
|
||||
ShardPath.deleteLeftoverShardDirectory(logger, nodeEnv, lock, indexSettings);
|
||||
path = ShardPath.loadShardPath(logger, nodeEnv, shardId, indexSettings);
|
||||
ShardPath.deleteLeftoverShardDirectory(logger, nodeEnv, lock, this.indexSettings);
|
||||
path = ShardPath.loadShardPath(logger, nodeEnv, shardId, this.indexSettings);
|
||||
} catch (Throwable t) {
|
||||
t.addSuppressed(ex);
|
||||
throw t;
|
||||
|
@ -260,7 +248,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
}
|
||||
dataPathToShardCount.put(dataPath, curCount+1);
|
||||
}
|
||||
path = ShardPath.selectNewPathForShard(nodeEnv, shardId, indexSettings, routing.getExpectedShardSize() == ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE ? getAvgShardSizeInBytes() : routing.getExpectedShardSize(),
|
||||
path = ShardPath.selectNewPathForShard(nodeEnv, shardId, this.indexSettings, routing.getExpectedShardSize() == ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE ? getAvgShardSizeInBytes() : routing.getExpectedShardSize(),
|
||||
dataPathToShardCount);
|
||||
logger.debug("{} creating using a new path [{}]", shardId, path);
|
||||
} else {
|
||||
|
@ -275,16 +263,15 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
// if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary.
|
||||
final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false ||
|
||||
(primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
|
||||
store = new Store(shardId, indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> indexServicesProvider.getIndicesQueryCache().onClose(shardId)));
|
||||
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> indexServicesProvider.getIndicesQueryCache().onClose(shardId)));
|
||||
if (useShadowEngine(primary, indexSettings)) {
|
||||
indexShard = new ShadowIndexShard(shardId, indexSettings, path, store, indexServicesProvider);
|
||||
indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexServicesProvider);
|
||||
} else {
|
||||
indexShard = new IndexShard(shardId, indexSettings, path, store, indexServicesProvider);
|
||||
indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexServicesProvider);
|
||||
}
|
||||
|
||||
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
||||
eventListener.afterIndexShardCreated(indexShard);
|
||||
settingsService.addListener(indexShard);
|
||||
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
|
||||
success = true;
|
||||
return indexShard;
|
||||
|
@ -301,7 +288,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
}
|
||||
|
||||
public synchronized void removeShard(int shardId, String reason) {
|
||||
final ShardId sId = new ShardId(index, shardId);
|
||||
final ShardId sId = new ShardId(index(), shardId);
|
||||
final IndexShard indexShard;
|
||||
if (shards.containsKey(shardId) == false) {
|
||||
return;
|
||||
|
@ -316,7 +303,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
|
||||
private void closeShard(String reason, ShardId sId, IndexShard indexShard, Store store, IndexEventListener listener) {
|
||||
final int shardId = sId.id();
|
||||
final Settings indexSettings = settingsService.getSettings();
|
||||
final Settings indexSettings = this.getIndexSettings().getSettings();
|
||||
try {
|
||||
try {
|
||||
listener.beforeIndexShardClosed(sId, indexShard, indexSettings);
|
||||
|
@ -324,7 +311,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
// this logic is tricky, we want to close the engine so we rollback the changes done to it
|
||||
// and close the shard so no operations are allowed to it
|
||||
if (indexShard != null) {
|
||||
settingsService.removeListener(indexShard);
|
||||
try {
|
||||
final boolean flushEngine = deleted.get() == false && closed.get(); // only flush we are we closed (closed index or shutdown) and if we are not deleted
|
||||
indexShard.close(reason, flushEngine);
|
||||
|
@ -348,7 +334,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
|
||||
private void onShardClose(ShardLock lock, boolean ownsShard) {
|
||||
if (deleted.get()) { // we remove that shards content if this index has been deleted
|
||||
final Settings indexSettings = settingsService.getSettings();
|
||||
final Settings indexSettings = this.getIndexSettings().getSettings();
|
||||
try {
|
||||
if (ownsShard) {
|
||||
try {
|
||||
|
@ -369,6 +355,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
return indexServicesProvider;
|
||||
}
|
||||
|
||||
public IndexSettings getIndexSettings() {
|
||||
return indexSettings;
|
||||
}
|
||||
|
||||
private class StoreCloseListener implements Store.OnClose {
|
||||
private final ShardId shardId;
|
||||
private final boolean ownsShard;
|
||||
|
@ -396,10 +386,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
}
|
||||
}
|
||||
|
||||
public Settings getIndexSettings() {
|
||||
return settingsService.getSettings();
|
||||
}
|
||||
|
||||
private static final class BitsetCacheListener implements BitsetFilterCache.Listener {
|
||||
final IndexService indexService;
|
||||
|
||||
|
@ -468,12 +454,12 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
return null;
|
||||
}
|
||||
final IndexQueryParserService indexQueryParser = queryParserService();
|
||||
final ImmutableOpenMap<String, AliasMetaData> aliases = this.indexMetaData.getAliases();
|
||||
final ImmutableOpenMap<String, AliasMetaData> aliases = indexSettings.getIndexMetaData().getAliases();
|
||||
if (aliasNames.length == 1) {
|
||||
AliasMetaData alias = aliases.get(aliasNames[0]);
|
||||
if (alias == null) {
|
||||
// This shouldn't happen unless alias disappeared after filteringAliases was called.
|
||||
throw new InvalidAliasNameException(index, aliasNames[0], "Unknown alias name was passed to alias Filter");
|
||||
throw new InvalidAliasNameException(index(), aliasNames[0], "Unknown alias name was passed to alias Filter");
|
||||
}
|
||||
return parse(alias, indexQueryParser);
|
||||
} else {
|
||||
|
@ -513,11 +499,24 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
}
|
||||
|
||||
public IndexMetaData getMetaData() {
|
||||
return indexMetaData;
|
||||
return indexSettings.getIndexMetaData();
|
||||
}
|
||||
|
||||
public void updateMetaData(IndexMetaData metadata) {
|
||||
this.indexMetaData = metadata;
|
||||
public synchronized void updateMetaData(final IndexMetaData metadata) {
|
||||
if (indexSettings.updateIndexMetaData(metadata)) {
|
||||
final Settings settings = indexSettings.getSettings();
|
||||
for (final IndexShard shard : this.shards.values()) {
|
||||
try {
|
||||
shard.onRefreshSettings(settings);
|
||||
} catch (Exception e) {
|
||||
logger.warn("[{}] failed to refresh shard settings", e, shard.shardId().id());
|
||||
}
|
||||
}
|
||||
try {
|
||||
indexStore.onRefreshSettings(settings);
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to refresh index store settings", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,190 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
/**
|
||||
* This class encapsulates all index level settings and handles settings updates.
|
||||
* It's created per index and available to all index level classes and allows them to retrieve
|
||||
* the latest updated settings instance. Classes that need to listen to settings updates can register
|
||||
* a settings consumer at index creation via {@link IndexModule#addIndexSettingsListener(Consumer)} that will
|
||||
* be called for each settings update.
|
||||
*/
|
||||
public final class IndexSettings {
|
||||
private final String uuid;
|
||||
private final List<Consumer<Settings>> updateListeners;
|
||||
private final Index index;
|
||||
private final Version version;
|
||||
private final ESLogger logger;
|
||||
private final String nodeName;
|
||||
private final Settings nodeSettings;
|
||||
private final int numberOfShards;
|
||||
private final boolean isShadowReplicaIndex;
|
||||
// volatile fields are updated via #updateIndexMetaData(IndexMetaData) under lock
|
||||
private volatile Settings settings;
|
||||
private volatile IndexMetaData indexMetaData;
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new {@link IndexSettings} instance. The given node settings will be merged with the settings in the metadata
|
||||
* while index level settings will overwrite node settings.
|
||||
*
|
||||
* @param indexMetaData the index metadata this settings object is associated with
|
||||
* @param nodeSettings the nodes settings this index is allocated on.
|
||||
* @param updateListeners a collection of listeners / consumers that should be notified if one or more settings are updated
|
||||
*/
|
||||
public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSettings, final Collection<Consumer<Settings>> updateListeners) {
|
||||
this.nodeSettings = nodeSettings;
|
||||
this.settings = Settings.builder().put(nodeSettings).put(indexMetaData.getSettings()).build();
|
||||
this.updateListeners = Collections.unmodifiableList(new ArrayList<>(updateListeners));
|
||||
this.index = new Index(indexMetaData.getIndex());
|
||||
version = Version.indexCreated(settings);
|
||||
uuid = settings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
|
||||
logger = Loggers.getLogger(getClass(), settings, index);
|
||||
nodeName = settings.get("name", "");
|
||||
this.indexMetaData = indexMetaData;
|
||||
numberOfShards = settings.getAsInt(IndexMetaData.SETTING_NUMBER_OF_SHARDS, null);
|
||||
isShadowReplicaIndex = IndexMetaData.isIndexUsingShadowReplicas(settings);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the settings for this index. These settings contain the node and index level settings where
|
||||
* settings that are specified on both index and node level are overwritten by the index settings.
|
||||
*/
|
||||
public Settings getSettings() { return settings; }
|
||||
|
||||
/**
|
||||
* Returns the index this settings object belongs to
|
||||
*/
|
||||
public Index getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the indexes UUID
|
||||
*/
|
||||
public String getUUID() {
|
||||
return uuid;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if the index has a custom data path
|
||||
*/
|
||||
public boolean hasCustomDataPath() {
|
||||
return NodeEnvironment.hasCustomDataPath(settings);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the version the index was created on.
|
||||
* @see Version#indexCreated(Settings)
|
||||
*/
|
||||
public Version getIndexVersionCreated() {
|
||||
return version;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current node name
|
||||
*/
|
||||
public String getNodeName() {
|
||||
return nodeName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current IndexMetaData for this index
|
||||
*/
|
||||
public IndexMetaData getIndexMetaData() {
|
||||
return indexMetaData;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of shards this index has.
|
||||
*/
|
||||
public int getNumberOfShards() { return numberOfShards; }
|
||||
|
||||
/**
|
||||
* Returns the number of replicas this index has.
|
||||
*/
|
||||
public int getNumberOfReplicas() { return settings.getAsInt(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, null); }
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff this index uses shadow replicas.
|
||||
* @see IndexMetaData#isIndexUsingShadowReplicas(Settings)
|
||||
*/
|
||||
public boolean isShadowReplicaIndex() {
|
||||
return isShadowReplicaIndex;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the node settings. The settings retured from {@link #getSettings()} are a merged version of the
|
||||
* index settings and the node settings where node settings are overwritten by index settings.
|
||||
*/
|
||||
public Settings getNodeSettings() {
|
||||
return nodeSettings;
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the settings and index metadata and notifies all registered settings consumers with the new settings iff at least one setting has changed.
|
||||
*
|
||||
* @return <code>true</code> iff any setting has been updated otherwise <code>false</code>.
|
||||
*/
|
||||
synchronized boolean updateIndexMetaData(IndexMetaData indexMetaData) {
|
||||
final Settings newSettings = indexMetaData.getSettings();
|
||||
if (Version.indexCreated(newSettings) != version) {
|
||||
throw new IllegalArgumentException("version mismatch on settings update expected: " + version + " but was: " + Version.indexCreated(newSettings));
|
||||
}
|
||||
final String newUUID = newSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
|
||||
if (newUUID.equals(getUUID()) == false) {
|
||||
throw new IllegalArgumentException("uuid mismatch on settings update expected: " + uuid + " but was: " + newUUID);
|
||||
}
|
||||
this.indexMetaData = indexMetaData;
|
||||
final Settings existingSettings = this.settings;
|
||||
if (existingSettings.getByPrefix(IndexMetaData.INDEX_SETTING_PREFIX).getAsMap().equals(newSettings.getByPrefix(IndexMetaData.INDEX_SETTING_PREFIX).getAsMap())) {
|
||||
// nothing to update, same settings
|
||||
return false;
|
||||
}
|
||||
final Settings mergedSettings = this.settings = Settings.builder().put(nodeSettings).put(newSettings).build();
|
||||
for (final Consumer<Settings> consumer : updateListeners) {
|
||||
try {
|
||||
consumer.accept(mergedSettings);
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to refresh index settings for [{}]", e, mergedSettings);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all settings update consumers
|
||||
*/
|
||||
List<Consumer<Settings>> getUpdateListeners() { // for testing
|
||||
return updateListeners;
|
||||
}
|
||||
}
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for ASCIIFoldingFilter.
|
||||
|
@ -34,8 +33,8 @@ public class ASCIIFoldingTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final boolean preserveOriginal;
|
||||
|
||||
@Inject
|
||||
public ASCIIFoldingTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ASCIIFoldingTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
preserveOriginal = settings.getAsBoolean("preserve_original", false);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,10 +19,8 @@
|
|||
|
||||
package org.elasticsearch.index.analysis;
|
||||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -31,8 +29,8 @@ public abstract class AbstractCharFilterFactory extends AbstractIndexComponent i
|
|||
|
||||
private final String name;
|
||||
|
||||
public AbstractCharFilterFactory(Index index, @IndexSettings Settings indexSettings, String name) {
|
||||
super(index, indexSettings);
|
||||
public AbstractCharFilterFactory(IndexSettings indexSettings, String name) {
|
||||
super(indexSettings);
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -38,14 +37,13 @@ public abstract class AbstractIndexAnalyzerProvider<T extends Analyzer> extends
|
|||
/**
|
||||
* Constructs a new analyzer component, with the index name and its settings and the analyzer name.
|
||||
*
|
||||
* @param index The index name
|
||||
* @param indexSettings The index settings
|
||||
* @param indexSettings the settings and the name of the index
|
||||
* @param name The analyzer name
|
||||
*/
|
||||
public AbstractIndexAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, String name, Settings settings) {
|
||||
super(index, indexSettings);
|
||||
public AbstractIndexAnalyzerProvider(IndexSettings indexSettings, String name, Settings settings) {
|
||||
super(indexSettings);
|
||||
this.name = name;
|
||||
this.version = Analysis.parseAnalysisVersion(indexSettings, settings, logger);
|
||||
this.version = Analysis.parseAnalysisVersion(this.indexSettings.getSettings(), settings, logger);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,8 +22,7 @@ package org.elasticsearch.index.analysis;
|
|||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -34,10 +33,10 @@ public abstract class AbstractTokenFilterFactory extends AbstractIndexComponent
|
|||
|
||||
protected final Version version;
|
||||
|
||||
public AbstractTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, String name, Settings settings) {
|
||||
super(index, indexSettings);
|
||||
public AbstractTokenFilterFactory(IndexSettings indexSettings, String name, Settings settings) {
|
||||
super(indexSettings);
|
||||
this.name = name;
|
||||
this.version = Analysis.parseAnalysisVersion(indexSettings, settings, logger);
|
||||
this.version = Analysis.parseAnalysisVersion(this.indexSettings.getSettings(), settings, logger);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,8 +22,7 @@ package org.elasticsearch.index.analysis;
|
|||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -35,10 +34,10 @@ public abstract class AbstractTokenizerFactory extends AbstractIndexComponent im
|
|||
protected final Version version;
|
||||
|
||||
|
||||
public AbstractTokenizerFactory(Index index, @IndexSettings Settings indexSettings, String name, Settings settings) {
|
||||
super(index, indexSettings);
|
||||
public AbstractTokenizerFactory(IndexSettings indexSettings, String name, Settings settings) {
|
||||
super(indexSettings);
|
||||
this.name = name;
|
||||
this.version = Analysis.parseAnalysisVersion(indexSettings, settings, logger);
|
||||
this.version = Analysis.parseAnalysisVersion(this.indexSettings.getSettings(), settings, logger);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -63,21 +63,13 @@ import org.elasticsearch.common.logging.ESLogger;
|
|||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
|
@ -86,7 +78,7 @@ import static java.util.Collections.unmodifiableMap;
|
|||
*/
|
||||
public class Analysis {
|
||||
|
||||
public static Version parseAnalysisVersion(@IndexSettings Settings indexSettings, Settings settings, ESLogger logger) {
|
||||
public static Version parseAnalysisVersion(Settings indexSettings, Settings settings, ESLogger logger) {
|
||||
// check for explicit version on the specific analyzer component
|
||||
String sVersion = settings.get("version");
|
||||
if (sVersion != null) {
|
||||
|
|
|
@ -27,9 +27,8 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.mapper.core.StringFieldMapper;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.indices.analysis.IndicesAnalysisService;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
@ -53,21 +52,21 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
private final NamedAnalyzer defaultSearchQuoteAnalyzer;
|
||||
|
||||
|
||||
public AnalysisService(Index index, Settings indexSettings) {
|
||||
this(index, indexSettings, null, null, null, null, null);
|
||||
public AnalysisService(IndexSettings indexSettings) {
|
||||
this(indexSettings, null, null, null, null, null);
|
||||
}
|
||||
|
||||
@Inject
|
||||
public AnalysisService(Index index, @IndexSettings Settings indexSettings, @Nullable IndicesAnalysisService indicesAnalysisService,
|
||||
public AnalysisService(IndexSettings indexSettings, @Nullable IndicesAnalysisService indicesAnalysisService,
|
||||
@Nullable Map<String, AnalyzerProviderFactory> analyzerFactoryFactories,
|
||||
@Nullable Map<String, TokenizerFactoryFactory> tokenizerFactoryFactories,
|
||||
@Nullable Map<String, CharFilterFactoryFactory> charFilterFactoryFactories,
|
||||
@Nullable Map<String, TokenFilterFactoryFactory> tokenFilterFactoryFactories) {
|
||||
super(index, indexSettings);
|
||||
Settings defaultSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.indexCreated(indexSettings)).build();
|
||||
super(indexSettings);
|
||||
Settings defaultSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, indexSettings.getIndexVersionCreated()).build();
|
||||
Map<String, TokenizerFactory> tokenizers = new HashMap<>();
|
||||
if (tokenizerFactoryFactories != null) {
|
||||
Map<String, Settings> tokenizersSettings = indexSettings.getGroups("index.analysis.tokenizer");
|
||||
Map<String, Settings> tokenizersSettings = this.indexSettings.getSettings().getGroups("index.analysis.tokenizer");
|
||||
for (Map.Entry<String, TokenizerFactoryFactory> entry : tokenizerFactoryFactories.entrySet()) {
|
||||
String tokenizerName = entry.getKey();
|
||||
TokenizerFactoryFactory tokenizerFactoryFactory = entry.getValue();
|
||||
|
@ -102,7 +101,7 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
|
||||
Map<String, CharFilterFactory> charFilters = new HashMap<>();
|
||||
if (charFilterFactoryFactories != null) {
|
||||
Map<String, Settings> charFiltersSettings = indexSettings.getGroups("index.analysis.char_filter");
|
||||
Map<String, Settings> charFiltersSettings = this.indexSettings.getSettings().getGroups("index.analysis.char_filter");
|
||||
for (Map.Entry<String, CharFilterFactoryFactory> entry : charFilterFactoryFactories.entrySet()) {
|
||||
String charFilterName = entry.getKey();
|
||||
CharFilterFactoryFactory charFilterFactoryFactory = entry.getValue();
|
||||
|
@ -137,7 +136,7 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
|
||||
Map<String, TokenFilterFactory> tokenFilters = new HashMap<>();
|
||||
if (tokenFilterFactoryFactories != null) {
|
||||
Map<String, Settings> tokenFiltersSettings = indexSettings.getGroups("index.analysis.filter");
|
||||
Map<String, Settings> tokenFiltersSettings = this.indexSettings.getSettings().getGroups("index.analysis.filter");
|
||||
for (Map.Entry<String, TokenFilterFactoryFactory> entry : tokenFilterFactoryFactories.entrySet()) {
|
||||
String tokenFilterName = entry.getKey();
|
||||
TokenFilterFactoryFactory tokenFilterFactoryFactory = entry.getValue();
|
||||
|
@ -172,7 +171,7 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
|
||||
Map<String, AnalyzerProvider> analyzerProviders = new HashMap<>();
|
||||
if (analyzerFactoryFactories != null) {
|
||||
Map<String, Settings> analyzersSettings = indexSettings.getGroups("index.analysis.analyzer");
|
||||
Map<String, Settings> analyzersSettings = this.indexSettings.getSettings().getGroups("index.analysis.analyzer");
|
||||
for (Map.Entry<String, AnalyzerProviderFactory> entry : analyzerFactoryFactories.entrySet()) {
|
||||
String analyzerName = entry.getKey();
|
||||
AnalyzerProviderFactory analyzerFactoryFactory = entry.getValue();
|
||||
|
@ -189,7 +188,7 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
if (indicesAnalysisService != null) {
|
||||
for (Map.Entry<String, PreBuiltAnalyzerProviderFactory> entry : indicesAnalysisService.analyzerProviderFactories().entrySet()) {
|
||||
String name = entry.getKey();
|
||||
Version indexVersion = Version.indexCreated(indexSettings);
|
||||
Version indexVersion = indexSettings.getIndexVersionCreated();
|
||||
if (!analyzerProviders.containsKey(name)) {
|
||||
analyzerProviders.put(name, entry.getValue().create(name, Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, indexVersion).build()));
|
||||
}
|
||||
|
@ -201,7 +200,7 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
}
|
||||
|
||||
if (!analyzerProviders.containsKey("default")) {
|
||||
analyzerProviders.put("default", new StandardAnalyzerProvider(index, indexSettings, null, "default", Settings.Builder.EMPTY_SETTINGS));
|
||||
analyzerProviders.put("default", new StandardAnalyzerProvider(indexSettings, null, "default", Settings.Builder.EMPTY_SETTINGS));
|
||||
}
|
||||
if (!analyzerProviders.containsKey("default_index")) {
|
||||
analyzerProviders.put("default_index", analyzerProviders.get("default"));
|
||||
|
@ -221,7 +220,7 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
* and 100 afterwards so we override the positionIncrementGap if it
|
||||
* doesn't match here.
|
||||
*/
|
||||
int overridePositionIncrementGap = StringFieldMapper.Defaults.positionIncrementGap(Version.indexCreated(indexSettings));
|
||||
int overridePositionIncrementGap = StringFieldMapper.Defaults.positionIncrementGap(indexSettings.getIndexVersionCreated());
|
||||
if (analyzerFactory instanceof CustomAnalyzerProvider) {
|
||||
((CustomAnalyzerProvider) analyzerFactory).build(this);
|
||||
/*
|
||||
|
@ -250,13 +249,13 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
}
|
||||
analyzers.put(analyzerFactory.name(), analyzer);
|
||||
analyzers.put(Strings.toCamelCase(analyzerFactory.name()), analyzer);
|
||||
String strAliases = indexSettings.get("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
|
||||
String strAliases = this.indexSettings.getSettings().get("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
|
||||
if (strAliases != null) {
|
||||
for (String alias : Strings.commaDelimitedListToStringArray(strAliases)) {
|
||||
analyzers.put(alias, analyzer);
|
||||
}
|
||||
}
|
||||
String[] aliases = indexSettings.getAsArray("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
|
||||
String[] aliases = this.indexSettings.getSettings().getAsArray("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
|
||||
for (String alias : aliases) {
|
||||
analyzers.put(alias, analyzer);
|
||||
}
|
||||
|
@ -267,11 +266,11 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
|
|||
throw new IllegalArgumentException("no default analyzer configured");
|
||||
}
|
||||
if (analyzers.containsKey("default_index")) {
|
||||
final Version createdVersion = Version.indexCreated(indexSettings);
|
||||
final Version createdVersion = indexSettings.getIndexVersionCreated();
|
||||
if (createdVersion.onOrAfter(Version.V_3_0_0)) {
|
||||
throw new IllegalArgumentException("setting [index.analysis.analyzer.default_index] is not supported anymore, use [index.analysis.analyzer.default] instead for index [" + index.getName() + "]");
|
||||
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());
|
||||
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;
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.tr.ApostropheFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link ApostropheFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ApostropheFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ApostropheFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ApostropheFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class ArabicAnalyzerProvider extends AbstractIndexAnalyzerProvider<Arabic
|
|||
private final ArabicAnalyzer arabicAnalyzer;
|
||||
|
||||
@Inject
|
||||
public ArabicAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ArabicAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
arabicAnalyzer = new ArabicAnalyzer(Analysis.parseStopWords(env, settings, ArabicAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
arabicAnalyzer.setVersion(version);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.ar.ArabicNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ArabicNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ArabicNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ArabicNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.ar.ArabicStemFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +32,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ArabicStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ArabicStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ArabicStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class ArmenianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Arme
|
|||
private final ArmenianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public ArmenianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ArmenianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new ArmenianAnalyzer(Analysis.parseStopWords(env, settings, ArmenianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class BasqueAnalyzerProvider extends AbstractIndexAnalyzerProvider<Basque
|
|||
private final BasqueAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public BasqueAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public BasqueAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new BasqueAnalyzer(Analysis.parseStopWords(env, settings, BasqueAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class BrazilianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Bra
|
|||
private final BrazilianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public BrazilianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public BrazilianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new BrazilianAnalyzer(Analysis.parseStopWords(env, settings, BrazilianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -26,8 +26,7 @@ import org.apache.lucene.analysis.util.CharArraySet;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -36,8 +35,8 @@ public class BrazilianStemTokenFilterFactory extends AbstractTokenFilterFactory
|
|||
private final CharArraySet exclusions;
|
||||
|
||||
@Inject
|
||||
public BrazilianStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public BrazilianStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.exclusions = Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class BulgarianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Bul
|
|||
private final BulgarianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public BulgarianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public BulgarianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new BulgarianAnalyzer(Analysis.parseStopWords(env, settings, BulgarianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.cjk.CJKBigramFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
|
@ -51,8 +50,8 @@ public final class CJKBigramFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final boolean outputUnigrams;
|
||||
|
||||
@Inject
|
||||
public CJKBigramFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CJKBigramFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
outputUnigrams = settings.getAsBoolean("output_unigrams", false);
|
||||
final String[] asArray = settings.getAsArray("ignored_scripts");
|
||||
Set<String> scripts = new HashSet<>(Arrays.asList("han", "hiragana", "katakana", "hangul"));
|
||||
|
|
|
@ -23,13 +23,13 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.analysis.cjk.CJKWidthFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
public final class CJKWidthFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public CJKWidthFilterFactory(Index index, Settings indexSettings, String name, Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CJKWidthFilterFactory(IndexSettings indexSettings, String name, Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class CatalanAnalyzerProvider extends AbstractIndexAnalyzerProvider<Catal
|
|||
private final CatalanAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public CatalanAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CatalanAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new CatalanAnalyzer(Analysis.parseStopWords(env, settings, CatalanAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Only for old indexes
|
||||
|
@ -34,8 +33,8 @@ public class ChineseAnalyzerProvider extends AbstractIndexAnalyzerProvider<Stand
|
|||
private final StandardAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public ChineseAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ChineseAnalyzerProvider(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
// old index: best effort
|
||||
analyzer = new StandardAnalyzer();
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class CjkAnalyzerProvider extends AbstractIndexAnalyzerProvider<CJKAnalyz
|
|||
private final CJKAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public CjkAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CjkAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
CharArraySet stopWords = Analysis.parseStopWords(env, settings, CJKAnalyzer.getDefaultStopSet());
|
||||
|
||||
analyzer = new CJKAnalyzer(stopWords);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.standard.ClassicFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link ClassicFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ClassicFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ClassicFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ClassicFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link ClassicTokenizer}
|
||||
|
@ -36,8 +35,8 @@ public class ClassicTokenizerFactory extends AbstractTokenizerFactory {
|
|||
private final int maxTokenLength;
|
||||
|
||||
@Inject
|
||||
public ClassicTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ClassicTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
maxTokenLength = settings.getAsInt("max_token_length", StandardAnalyzer.DEFAULT_MAX_TOKEN_LENGTH);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,16 +19,15 @@
|
|||
|
||||
package org.elasticsearch.index.analysis;
|
||||
|
||||
import org.apache.lucene.analysis.util.CharArraySet;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
|
||||
import org.apache.lucene.analysis.commongrams.CommonGramsQueryFilter;
|
||||
import org.apache.lucene.analysis.util.CharArraySet;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -43,8 +42,8 @@ public class CommonGramsTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final boolean queryMode;
|
||||
|
||||
@Inject
|
||||
public CommonGramsTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CommonGramsTokenFilterFactory(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.ignoreCase = settings.getAsBoolean("ignore_case", false);
|
||||
this.queryMode = settings.getAsBoolean("query_mode", false);
|
||||
this.words = Analysis.parseCommonWords(env, settings, null, ignoreCase);
|
||||
|
|
|
@ -23,9 +23,8 @@ import org.elasticsearch.Version;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.mapper.core.StringFieldMapper;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -41,9 +40,9 @@ public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider<Custom
|
|||
private CustomAnalyzer customAnalyzer;
|
||||
|
||||
@Inject
|
||||
public CustomAnalyzerProvider(Index index, @IndexSettings Settings indexSettings,
|
||||
public CustomAnalyzerProvider(IndexSettings indexSettings,
|
||||
@Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
super(indexSettings, name, settings);
|
||||
this.analyzerSettings = settings;
|
||||
}
|
||||
|
||||
|
@ -78,10 +77,10 @@ public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider<Custom
|
|||
tokenFilters.add(tokenFilter);
|
||||
}
|
||||
|
||||
int positionIncrementGap = StringFieldMapper.Defaults.positionIncrementGap(Version.indexCreated(indexSettings));
|
||||
int positionIncrementGap = StringFieldMapper.Defaults.positionIncrementGap(indexSettings.getIndexVersionCreated());
|
||||
|
||||
if (analyzerSettings.getAsMap().containsKey("position_offset_gap")){
|
||||
if (Version.indexCreated(indexSettings).before(Version.V_2_0_0)){
|
||||
if (indexSettings.getIndexVersionCreated().before(Version.V_2_0_0)){
|
||||
if (analyzerSettings.getAsMap().containsKey("position_increment_gap")){
|
||||
throw new IllegalArgumentException("Custom Analyzer [" + name() +
|
||||
"] defined both [position_offset_gap] and [position_increment_gap], use only [position_increment_gap]");
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class CzechAnalyzerProvider extends AbstractIndexAnalyzerProvider<CzechAn
|
|||
private final CzechAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public CzechAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CzechAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new CzechAnalyzer(Analysis.parseStopWords(env, settings, CzechAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -23,14 +23,13 @@ import org.apache.lucene.analysis.cz.CzechStemFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
public class CzechStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public CzechStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public CzechStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class DanishAnalyzerProvider extends AbstractIndexAnalyzerProvider<Danish
|
|||
private final DanishAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public DanishAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public DanishAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new DanishAnalyzer(Analysis.parseStopWords(env, settings, DanishAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.analysis.core.DecimalDigitFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link DecimalDigitFilter}
|
||||
|
@ -31,8 +31,8 @@ import org.elasticsearch.index.Index;
|
|||
public final class DecimalDigitFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public DecimalDigitFilterFactory(Index index, Settings indexSettings, String name, Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public DecimalDigitFilterFactory(IndexSettings indexSettings, String name, Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -43,9 +42,9 @@ public class DelimitedPayloadTokenFilterFactory extends AbstractTokenFilterFacto
|
|||
PayloadEncoder encoder;
|
||||
|
||||
@Inject
|
||||
public DelimitedPayloadTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name,
|
||||
public DelimitedPayloadTokenFilterFactory(IndexSettings indexSettings, Environment env, @Assisted String name,
|
||||
@Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
super(indexSettings, name, settings);
|
||||
String delimiterConf = settings.get(DELIMITER);
|
||||
if (delimiterConf != null) {
|
||||
delimiter = delimiterConf.charAt(0);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class DutchAnalyzerProvider extends AbstractIndexAnalyzerProvider<DutchAn
|
|||
private final DutchAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public DutchAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public DutchAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new DutchAnalyzer(Analysis.parseStopWords(env, settings, DutchAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -26,8 +26,7 @@ import org.apache.lucene.analysis.util.CharArraySet;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.tartarus.snowball.ext.DutchStemmer;
|
||||
|
||||
/**
|
||||
|
@ -38,8 +37,8 @@ public class DutchStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final CharArraySet exclusions;
|
||||
|
||||
@Inject
|
||||
public DutchStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public DutchStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.exclusions = Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET);
|
||||
}
|
||||
|
||||
|
|
|
@ -28,8 +28,7 @@ import org.apache.lucene.util.Version;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -48,12 +47,12 @@ public class EdgeNGramTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private org.elasticsearch.Version esVersion;
|
||||
|
||||
@Inject
|
||||
public EdgeNGramTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public EdgeNGramTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.minGram = settings.getAsInt("min_gram", NGramTokenFilter.DEFAULT_MIN_NGRAM_SIZE);
|
||||
this.maxGram = settings.getAsInt("max_gram", NGramTokenFilter.DEFAULT_MAX_NGRAM_SIZE);
|
||||
this.side = parseSide(settings.get("side", "front"));
|
||||
this.esVersion = org.elasticsearch.Version.indexCreated(indexSettings);
|
||||
this.esVersion = org.elasticsearch.Version.indexCreated(indexSettings.getSettings());
|
||||
}
|
||||
|
||||
static int parseSide(String side) {
|
||||
|
|
|
@ -24,12 +24,10 @@ import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
|
|||
import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenizer;
|
||||
import org.apache.lucene.analysis.ngram.NGramTokenizer;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import static org.elasticsearch.index.analysis.NGramTokenizerFactory.parseTokenChars;
|
||||
|
||||
|
@ -51,13 +49,13 @@ public class EdgeNGramTokenizerFactory extends AbstractTokenizerFactory {
|
|||
|
||||
|
||||
@Inject
|
||||
public EdgeNGramTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public EdgeNGramTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.minGram = settings.getAsInt("min_gram", NGramTokenizer.DEFAULT_MIN_NGRAM_SIZE);
|
||||
this.maxGram = settings.getAsInt("max_gram", NGramTokenizer.DEFAULT_MAX_NGRAM_SIZE);
|
||||
this.side = Lucene43EdgeNGramTokenizer.Side.getSide(settings.get("side", Lucene43EdgeNGramTokenizer.DEFAULT_SIDE.getLabel()));
|
||||
this.matcher = parseTokenChars(settings.getAsArray("token_chars"));
|
||||
this.esVersion = org.elasticsearch.Version.indexCreated(indexSettings);
|
||||
this.esVersion = indexSettings.getIndexVersionCreated();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,8 +26,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -37,8 +36,8 @@ public class ElisionTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final CharArraySet articles;
|
||||
|
||||
@Inject
|
||||
public ElisionTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ElisionTokenFilterFactory(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.articles = Analysis.parseArticles(env, settings);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class EnglishAnalyzerProvider extends AbstractIndexAnalyzerProvider<Engli
|
|||
private final EnglishAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public EnglishAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public EnglishAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new EnglishAnalyzer(Analysis.parseStopWords(env, settings, EnglishAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class FinnishAnalyzerProvider extends AbstractIndexAnalyzerProvider<Finni
|
|||
private final FinnishAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public FinnishAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public FinnishAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new FinnishAnalyzer(Analysis.parseStopWords(env, settings, FinnishAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class FrenchAnalyzerProvider extends AbstractIndexAnalyzerProvider<French
|
|||
private final FrenchAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public FrenchAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public FrenchAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new FrenchAnalyzer(Analysis.parseStopWords(env, settings, FrenchAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -21,14 +21,13 @@ package org.elasticsearch.index.analysis;
|
|||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
|
||||
import org.apache.lucene.analysis.util.CharArraySet;
|
||||
import org.apache.lucene.analysis.snowball.SnowballFilter;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.tartarus.snowball.ext.FrenchStemmer;
|
||||
import org.apache.lucene.analysis.util.CharArraySet;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.tartarus.snowball.ext.FrenchStemmer;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -38,8 +37,8 @@ public class FrenchStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final CharArraySet exclusions;
|
||||
|
||||
@Inject
|
||||
public FrenchStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public FrenchStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.exclusions = Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class GalicianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Gali
|
|||
private final GalicianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public GalicianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public GalicianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new GalicianAnalyzer(Analysis.parseStopWords(env, settings, GalicianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class GermanAnalyzerProvider extends AbstractIndexAnalyzerProvider<German
|
|||
private final GermanAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public GermanAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public GermanAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new GermanAnalyzer(Analysis.parseStopWords(env, settings, GermanAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.de.GermanNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link GermanNormalizationFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class GermanNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public GermanNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public GermanNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,8 +26,7 @@ import org.apache.lucene.analysis.util.CharArraySet;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -37,8 +36,8 @@ public class GermanStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final CharArraySet exclusions;
|
||||
|
||||
@Inject
|
||||
public GermanStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public GermanStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.exclusions = Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -35,8 +34,8 @@ public class GreekAnalyzerProvider extends AbstractIndexAnalyzerProvider<GreekAn
|
|||
private final GreekAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public GreekAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public GreekAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new GreekAnalyzer(Analysis.parseStopWords(env, settings, GreekAnalyzer.getDefaultStopSet()));
|
||||
analyzer.setVersion(version);
|
||||
}
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class HindiAnalyzerProvider extends AbstractIndexAnalyzerProvider<HindiAn
|
|||
private final HindiAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public HindiAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public HindiAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new HindiAnalyzer(Analysis.parseStopWords(env, settings, HindiAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.hi.HindiNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link HindiNormalizationFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class HindiNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public HindiNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public HindiNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.charfilter.HTMLStripCharFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.io.Reader;
|
||||
import java.util.Set;
|
||||
|
@ -36,8 +35,8 @@ public class HtmlStripCharFilterFactory extends AbstractCharFilterFactory {
|
|||
private final Set<String> escapedTags;
|
||||
|
||||
@Inject
|
||||
public HtmlStripCharFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name);
|
||||
public HtmlStripCharFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name);
|
||||
String[] escapedTags = settings.getAsArray("escaped_tags");
|
||||
if (escapedTags.length > 0) {
|
||||
this.escapedTags = unmodifiableSet(newHashSet(escapedTags));
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class HungarianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Hun
|
|||
private final HungarianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public HungarianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public HungarianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new HungarianAnalyzer(Analysis.parseStopWords(env, settings, HungarianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.hunspell.HunspellStemFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.indices.analysis.HunspellService;
|
||||
|
||||
import java.util.Locale;
|
||||
|
@ -38,8 +37,8 @@ public class HunspellTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final boolean longestOnly;
|
||||
|
||||
@Inject
|
||||
public HunspellTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings, HunspellService hunspellService) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public HunspellTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings, HunspellService hunspellService) {
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
String locale = settings.get("locale", settings.get("language", settings.get("lang", null)));
|
||||
if (locale == null) {
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.in.IndicNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link IndicNormalizationFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class IndicNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public IndicNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public IndicNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class IndonesianAnalyzerProvider extends AbstractIndexAnalyzerProvider<In
|
|||
private final IndonesianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public IndonesianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public IndonesianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new IndonesianAnalyzer(Analysis.parseStopWords(env, settings, IndonesianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Provider for {@link IrishAnalyzer}
|
||||
|
@ -36,8 +35,8 @@ public class IrishAnalyzerProvider extends AbstractIndexAnalyzerProvider<IrishAn
|
|||
private final IrishAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public IrishAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public IrishAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new IrishAnalyzer(Analysis.parseStopWords(env, settings, IrishAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class ItalianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Itali
|
|||
private final ItalianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public ItalianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ItalianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new ItalianAnalyzer(Analysis.parseStopWords(env, settings, ItalianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -24,14 +24,13 @@ import org.apache.lucene.analysis.en.KStemFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
public class KStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public KStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public KStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
|
@ -48,9 +47,9 @@ public class KeepTypesFilterFactory extends AbstractTokenFilterFactory {
|
|||
private static final String KEEP_TYPES_KEY = "types";
|
||||
|
||||
@Inject
|
||||
public KeepTypesFilterFactory(Index index, @IndexSettings Settings indexSettings,
|
||||
public KeepTypesFilterFactory(IndexSettings indexSettings,
|
||||
Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
final String[] arrayKeepTypes = settings.getAsArray(KEEP_TYPES_KEY, null);
|
||||
if ((arrayKeepTypes == null)) {
|
||||
|
|
|
@ -28,8 +28,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* A {@link TokenFilterFactory} for {@link KeepWordFilter}. This filter only
|
||||
|
@ -62,9 +61,9 @@ public class KeepWordFilterFactory extends AbstractTokenFilterFactory {
|
|||
private static final String ENABLE_POS_INC_KEY = "enable_position_increments";
|
||||
|
||||
@Inject
|
||||
public KeepWordFilterFactory(Index index, @IndexSettings Settings indexSettings,
|
||||
public KeepWordFilterFactory(IndexSettings indexSettings,
|
||||
Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
final String[] arrayKeepWords = settings.getAsArray(KEEP_WORDS_KEY, null);
|
||||
final String keepWordsPath = settings.get(KEEP_WORDS_PATH_KEY, null);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.core.KeywordAnalyzer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -34,8 +33,8 @@ public class KeywordAnalyzerProvider extends AbstractIndexAnalyzerProvider<Keywo
|
|||
private final KeywordAnalyzer keywordAnalyzer;
|
||||
|
||||
@Inject
|
||||
public KeywordAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public KeywordAnalyzerProvider(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.keywordAnalyzer = new KeywordAnalyzer();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.elasticsearch.index.analysis;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
|
||||
import org.apache.lucene.analysis.util.CharArraySet;
|
||||
|
@ -28,8 +26,9 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
@AnalysisSettingsRequired
|
||||
public class KeywordMarkerTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
@ -37,8 +36,8 @@ public class KeywordMarkerTokenFilterFactory extends AbstractTokenFilterFactory
|
|||
private final CharArraySet keywordLookup;
|
||||
|
||||
@Inject
|
||||
public KeywordMarkerTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public KeywordMarkerTokenFilterFactory(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
boolean ignoreCase = settings.getAsBoolean("ignore_case", false);
|
||||
Set<?> rules = Analysis.getWordSet(env, settings, "keywords");
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.core.KeywordTokenizer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -35,8 +34,8 @@ public class KeywordTokenizerFactory extends AbstractTokenizerFactory {
|
|||
private final int bufferSize;
|
||||
|
||||
@Inject
|
||||
public KeywordTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public KeywordTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
bufferSize = settings.getAsInt("buffer_size", 256);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class LatvianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Latvi
|
|||
private final LatvianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public LatvianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LatvianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new LatvianAnalyzer(Analysis.parseStopWords(env, settings, LatvianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -26,8 +26,7 @@ import org.apache.lucene.util.Version;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -40,8 +39,8 @@ public class LengthTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private static final String ENABLE_POS_INC_KEY = "enable_position_increments";
|
||||
|
||||
@Inject
|
||||
public LengthTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LengthTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
min = settings.getAsInt("min", 0);
|
||||
max = settings.getAsInt("max", Integer.MAX_VALUE);
|
||||
if (version.onOrAfter(Version.LUCENE_4_4) && settings.get(ENABLE_POS_INC_KEY) != null) {
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.core.LetterTokenizer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +32,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class LetterTokenizerFactory extends AbstractTokenizerFactory {
|
||||
|
||||
@Inject
|
||||
public LetterTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LetterTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -40,8 +39,8 @@ public class LimitTokenCountFilterFactory extends AbstractTokenFilterFactory {
|
|||
final boolean consumeAllTokens;
|
||||
|
||||
@Inject
|
||||
public LimitTokenCountFilterFactory(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LimitTokenCountFilterFactory(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.maxTokenCount = settings.getAsInt("max_token_count", DEFAULT_MAX_TOKEN_COUNT);
|
||||
this.consumeAllTokens = settings.getAsBoolean("consume_all_tokens", DEFAULT_CONSUME_ALL_TOKENS);
|
||||
}
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Provider for {@link LithuanianAnalyzer}
|
||||
|
@ -36,8 +35,8 @@ public class LithuanianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Li
|
|||
private final LithuanianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public LithuanianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LithuanianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new LithuanianAnalyzer(Analysis.parseStopWords(env, settings, LithuanianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -27,8 +27,7 @@ import org.apache.lucene.analysis.tr.TurkishLowerCaseFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link LowerCaseFilter} and some language-specific variants
|
||||
|
@ -44,8 +43,8 @@ public class LowerCaseTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final String lang;
|
||||
|
||||
@Inject
|
||||
public LowerCaseTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LowerCaseTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.lang = settings.get("language", null);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.core.LowerCaseTokenizer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +32,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class LowerCaseTokenizerFactory extends AbstractTokenizerFactory {
|
||||
|
||||
@Inject
|
||||
public LowerCaseTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public LowerCaseTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.io.Reader;
|
||||
import java.util.List;
|
||||
|
@ -39,8 +38,8 @@ public class MappingCharFilterFactory extends AbstractCharFilterFactory {
|
|||
private final NormalizeCharMap normMap;
|
||||
|
||||
@Inject
|
||||
public MappingCharFilterFactory(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name);
|
||||
public MappingCharFilterFactory(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name);
|
||||
|
||||
List<String> rules = Analysis.getWordList(env, settings, "mappings");
|
||||
if (rules == null) {
|
||||
|
|
|
@ -20,14 +20,13 @@
|
|||
package org.elasticsearch.index.analysis;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
|
||||
import org.apache.lucene.analysis.ngram.Lucene43NGramTokenFilter;
|
||||
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -41,8 +40,8 @@ public class NGramTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
|
||||
|
||||
@Inject
|
||||
public NGramTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public NGramTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.minGram = settings.getAsInt("min_gram", NGramTokenFilter.DEFAULT_MIN_NGRAM_SIZE);
|
||||
this.maxGram = settings.getAsInt("max_gram", NGramTokenFilter.DEFAULT_MAX_NGRAM_SIZE);
|
||||
}
|
||||
|
|
|
@ -26,8 +26,7 @@ import org.apache.lucene.util.Version;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.Modifier;
|
||||
|
@ -90,12 +89,12 @@ public class NGramTokenizerFactory extends AbstractTokenizerFactory {
|
|||
}
|
||||
|
||||
@Inject
|
||||
public NGramTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public NGramTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.minGram = settings.getAsInt("min_gram", NGramTokenizer.DEFAULT_MIN_NGRAM_SIZE);
|
||||
this.maxGram = settings.getAsInt("max_gram", NGramTokenizer.DEFAULT_MAX_NGRAM_SIZE);
|
||||
this.matcher = parseTokenChars(settings.getAsArray("token_chars"));
|
||||
this.esVersion = org.elasticsearch.Version.indexCreated(indexSettings);
|
||||
this.esVersion = indexSettings.getIndexVersionCreated();
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class NorwegianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Nor
|
|||
private final NorwegianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public NorwegianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public NorwegianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new NorwegianAnalyzer(Analysis.parseStopWords(env, settings, NorwegianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.apache.lucene.analysis.path.ReversePathHierarchyTokenizer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
public class PathHierarchyTokenizerFactory extends AbstractTokenizerFactory {
|
||||
|
||||
|
@ -38,8 +37,8 @@ public class PathHierarchyTokenizerFactory extends AbstractTokenizerFactory {
|
|||
private final boolean reverse;
|
||||
|
||||
@Inject
|
||||
public PathHierarchyTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PathHierarchyTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
bufferSize = settings.getAsInt("buffer_size", 1024);
|
||||
String delimiter = settings.get("delimiter");
|
||||
if (delimiter == null) {
|
||||
|
|
|
@ -28,8 +28,7 @@ import org.elasticsearch.common.inject.assistedinject.Assisted;
|
|||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -41,10 +40,10 @@ public class PatternAnalyzerProvider extends AbstractIndexAnalyzerProvider<Analy
|
|||
private final PatternAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public PatternAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PatternAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
Version esVersion = Version.indexCreated(indexSettings);
|
||||
Version esVersion = indexSettings.getIndexVersionCreated();
|
||||
final CharArraySet defaultStopwords;
|
||||
if (esVersion.onOrAfter(Version.V_1_0_0_RC1)) {
|
||||
defaultStopwords = CharArraySet.EMPTY_SET;
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.apache.lucene.analysis.pattern.PatternCaptureGroupTokenFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -38,9 +37,9 @@ public class PatternCaptureGroupTokenFilterFactory extends AbstractTokenFilterFa
|
|||
private static final String PRESERVE_ORIG_KEY = "preserve_original";
|
||||
|
||||
@Inject
|
||||
public PatternCaptureGroupTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name,
|
||||
public PatternCaptureGroupTokenFilterFactory(IndexSettings indexSettings, @Assisted String name,
|
||||
@Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
super(indexSettings, name, settings);
|
||||
String[] regexes = settings.getAsArray(PATTERNS_KEY, null, false);
|
||||
if (regexes == null) {
|
||||
throw new IllegalArgumentException("required setting '" + PATTERNS_KEY + "' is missing for token filter [" + name + "]");
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.io.Reader;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -36,8 +35,8 @@ public class PatternReplaceCharFilterFactory extends AbstractCharFilterFactory {
|
|||
private final String replacement;
|
||||
|
||||
@Inject
|
||||
public PatternReplaceCharFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name);
|
||||
public PatternReplaceCharFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name);
|
||||
|
||||
if (!Strings.hasLength(settings.get("pattern"))) {
|
||||
throw new IllegalArgumentException("pattern is missing for [" + name + "] char filter of type 'pattern_replace'");
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -38,8 +37,8 @@ public class PatternReplaceTokenFilterFactory extends AbstractTokenFilterFactory
|
|||
private final boolean all;
|
||||
|
||||
@Inject
|
||||
public PatternReplaceTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PatternReplaceTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
String sPattern = settings.get("pattern", null);
|
||||
if (sPattern == null) {
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -36,8 +35,8 @@ public class PatternTokenizerFactory extends AbstractTokenizerFactory {
|
|||
private final int group;
|
||||
|
||||
@Inject
|
||||
public PatternTokenizerFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PatternTokenizerFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
|
||||
String sPattern = settings.get("pattern", "\\W+" /*PatternAnalyzer.NON_WORD_PATTERN*/);
|
||||
if (sPattern == null) {
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -35,8 +34,8 @@ public class PersianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Persi
|
|||
private final PersianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public PersianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PersianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new PersianAnalyzer(Analysis.parseStopWords(env, settings, PersianAnalyzer.getDefaultStopSet()));
|
||||
analyzer.setVersion(version);
|
||||
}
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.fa.PersianNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class PersianNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public PersianNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PersianNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.en.PorterStemFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +32,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class PorterStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public PorterStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PorterStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class PortugueseAnalyzerProvider extends AbstractIndexAnalyzerProvider<Po
|
|||
private final PortugueseAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public PortugueseAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public PortugueseAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new PortugueseAnalyzer(Analysis.parseStopWords(env, settings, PortugueseAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.reverse.ReverseStringFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +32,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ReverseTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ReverseTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ReverseTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class RomanianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Roma
|
|||
private final RomanianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public RomanianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public RomanianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new RomanianAnalyzer(Analysis.parseStopWords(env, settings, RomanianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -36,8 +35,8 @@ public class RussianAnalyzerProvider extends AbstractIndexAnalyzerProvider<Russi
|
|||
private final RussianAnalyzer analyzer;
|
||||
|
||||
@Inject
|
||||
public RussianAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public RussianAnalyzerProvider(IndexSettings indexSettings, Environment env, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
analyzer = new RussianAnalyzer(Analysis.parseStopWords(env, settings, RussianAnalyzer.getDefaultStopSet()),
|
||||
Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET));
|
||||
analyzer.setVersion(version);
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.snowball.SnowballFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +32,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class RussianStemTokenFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public RussianStemTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public RussianStemTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.miscellaneous.ScandinavianFoldingFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link ScandinavianFoldingFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ScandinavianFoldingFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ScandinavianFoldingFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ScandinavianFoldingFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.miscellaneous.ScandinavianNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
* Factory for {@link ScandinavianNormalizationFilter}
|
||||
|
@ -32,8 +31,8 @@ import org.elasticsearch.index.settings.IndexSettings;
|
|||
public class ScandinavianNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public ScandinavianNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ScandinavianNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,16 +24,15 @@ import org.apache.lucene.analysis.sr.SerbianNormalizationFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class SerbianNormalizationFilterFactory extends AbstractTokenFilterFactory {
|
||||
|
||||
@Inject
|
||||
public SerbianNormalizationFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public SerbianNormalizationFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.lucene.analysis.shingle.ShingleFilter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -35,8 +34,8 @@ public class ShingleTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
private final Factory factory;
|
||||
|
||||
@Inject
|
||||
public ShingleTokenFilterFactory(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public ShingleTokenFilterFactory(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
Integer maxShingleSize = settings.getAsInt("max_shingle_size", ShingleFilter.DEFAULT_MAX_SHINGLE_SIZE);
|
||||
Integer minShingleSize = settings.getAsInt("min_shingle_size", ShingleFilter.DEFAULT_MIN_SHINGLE_SIZE);
|
||||
Boolean outputUnigrams = settings.getAsBoolean("output_unigrams", true);
|
||||
|
|
|
@ -23,8 +23,7 @@ import org.apache.lucene.analysis.core.SimpleAnalyzer;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -34,8 +33,8 @@ public class SimpleAnalyzerProvider extends AbstractIndexAnalyzerProvider<Simple
|
|||
private final SimpleAnalyzer simpleAnalyzer;
|
||||
|
||||
@Inject
|
||||
public SimpleAnalyzerProvider(Index index, @IndexSettings Settings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(index, indexSettings, name, settings);
|
||||
public SimpleAnalyzerProvider(IndexSettings indexSettings, @Assisted String name, @Assisted Settings settings) {
|
||||
super(indexSettings, name, settings);
|
||||
this.simpleAnalyzer = new SimpleAnalyzer();
|
||||
this.simpleAnalyzer.setVersion(version);
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue