Merge branch 'master' into feature/aggs-refactoring

This commit is contained in:
Colin Goodheart-Smithe 2016-01-12 12:59:52 +00:00
commit f429cb08d7
70 changed files with 1072 additions and 1152 deletions

View File

@ -152,7 +152,6 @@ public class ClusterModule extends AbstractModule {
registerIndexDynamicSetting(PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS, Validator.EMPTY);
registerIndexDynamicSetting(EngineConfig.INDEX_GC_DELETES_SETTING, Validator.TIME);
registerIndexDynamicSetting(IndexShard.INDEX_FLUSH_ON_CLOSE, Validator.BOOLEAN);
registerIndexDynamicSetting(EngineConfig.INDEX_VERSION_MAP_SIZE, Validator.BYTES_SIZE_OR_PERCENTAGE);
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN, Validator.TIME);
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO, Validator.TIME);
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG, Validator.TIME);

View File

@ -40,7 +40,7 @@ import static org.elasticsearch.common.Strings.cleanPath;
* The environment of where things exists.
*/
@SuppressForbidden(reason = "configures paths for the system")
// TODO: move PathUtils to be package-private here instead of
// TODO: move PathUtils to be package-private here instead of
// public+forbidden api!
public class Environment {
@ -72,7 +72,7 @@ public class Environment {
/** Path to the PID file (can be null if no PID file is configured) **/
private final Path pidFile;
/** Path to the temporary file directory used by the JDK */
private final Path tmpFile = PathUtils.get(System.getProperty("java.io.tmpdir"));
@ -292,7 +292,7 @@ public class Environment {
public Path pidFile() {
return pidFile;
}
/** Path to the default temp directory used by the JDK */
public Path tmpFile() {
return tmpFile;
@ -317,7 +317,7 @@ public class Environment {
public static FileStore getFileStore(Path path) throws IOException {
return ESFileStore.getMatchingFileStore(path, fileStores);
}
/**
* Returns true if the path is writable.
* Acts just like {@link Files#isWritable(Path)}, except won't

View File

@ -125,18 +125,6 @@ final class CompositeIndexEventListener implements IndexEventListener {
}
}
@Override
public void onShardActive(IndexShard indexShard) {
for (IndexEventListener listener : listeners) {
try {
listener.onShardActive(indexShard);
} catch (Throwable t) {
logger.warn("[{}] failed to invoke on shard active callback", t, indexShard.shardId().getId());
throw t;
}
}
}
@Override
public void indexShardStateChanged(IndexShard indexShard, @Nullable IndexShardState previousState, IndexShardState currentState, @Nullable String reason) {
for (IndexEventListener listener : listeners) {

View File

@ -29,11 +29,13 @@ import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.shard.IndexingOperationListener;
import org.elasticsearch.index.similarity.BM25SimilarityProvider;
import org.elasticsearch.index.similarity.SimilarityProvider;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.mapper.MapperRegistry;
@ -241,7 +243,8 @@ public final class IndexModule {
IndexSearcherWrapper newWrapper(final IndexService indexService);
}
public IndexService newIndexService(NodeEnvironment environment, IndexService.ShardStoreDeleter shardStoreDeleter, NodeServicesProvider servicesProvider, MapperRegistry mapperRegistry) throws IOException {
public IndexService newIndexService(NodeEnvironment environment, IndexService.ShardStoreDeleter shardStoreDeleter, NodeServicesProvider servicesProvider, MapperRegistry mapperRegistry,
IndexingOperationListener... listeners) throws IOException {
final IndexSettings settings = indexSettings.newWithListener(settingsConsumers);
IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get();
IndexEventListener eventListener = freeze();
@ -263,6 +266,6 @@ public final class IndexModule {
final BiFunction<IndexSettings, IndicesQueryCache, QueryCache> queryCacheProvider = queryCaches.get(queryCacheType);
final QueryCache queryCache = queryCacheProvider.apply(settings, servicesProvider.getIndicesQueryCache());
return new IndexService(settings, environment, new SimilarityService(settings, similarities), shardStoreDeleter, analysisRegistry, engineFactory.get(),
servicesProvider, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry);
servicesProvider, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, listeners);
}
}

View File

@ -19,6 +19,17 @@
package org.elasticsearch.index;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
@ -51,6 +62,7 @@ import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexingOperationListener;
import org.elasticsearch.index.shard.ShadowIndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
@ -64,17 +76,6 @@ import org.elasticsearch.indices.InvalidAliasNameException;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
@ -102,6 +103,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
private final AtomicBoolean deleted = new AtomicBoolean(false);
private final IndexSettings indexSettings;
private final IndexingSlowLog slowLog;
private final IndexingOperationListener[] listeners;
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
SimilarityService similarityService,
@ -113,7 +115,8 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
IndexStore indexStore,
IndexEventListener eventListener,
IndexModule.IndexSearcherWrapperFactory wrapperFactory,
MapperRegistry mapperRegistry) throws IOException {
MapperRegistry mapperRegistry,
IndexingOperationListener... listenersIn) throws IOException {
super(indexSettings);
this.indexSettings = indexSettings;
this.analysisService = registry.build(indexSettings);
@ -132,6 +135,11 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
// initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE
this.searcherWrapper = wrapperFactory.newWrapper(this);
this.slowLog = new IndexingSlowLog(indexSettings.getSettings());
// Add our slowLog to the incoming IndexingOperationListeners:
this.listeners = new IndexingOperationListener[1+listenersIn.length];
this.listeners[0] = slowLog;
System.arraycopy(listenersIn, 0, this.listeners, 1, listenersIn.length);
}
public int numberOfShards() {
@ -296,7 +304,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
if (useShadowEngine(primary, indexSettings)) {
indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider); // no indexing listeners - shadow engines don't index
} else {
indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, slowLog);
indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, listeners);
}
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");

View File

@ -20,9 +20,7 @@
package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.ngram.Lucene43NGramTokenizer;
import org.apache.lucene.analysis.ngram.NGramTokenizer;
import org.apache.lucene.util.Version;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
@ -43,7 +41,6 @@ public class NGramTokenizerFactory extends AbstractTokenizerFactory {
private final int minGram;
private final int maxGram;
private final CharMatcher matcher;
private org.elasticsearch.Version esVersion;
static final Map<String, CharMatcher> MATCHERS;
@ -92,30 +89,19 @@ public class NGramTokenizerFactory extends AbstractTokenizerFactory {
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 = indexSettings.getIndexVersionCreated();
}
@SuppressWarnings("deprecation")
@Override
public Tokenizer create() {
if (version.onOrAfter(Version.LUCENE_4_3) && esVersion.onOrAfter(org.elasticsearch.Version.V_0_90_2)) {
/*
* We added this in 0.90.2 but 0.90.1 used LUCENE_43 already so we can not rely on the lucene version.
* Yet if somebody uses 0.90.2 or higher with a prev. lucene version we should also use the deprecated version.
*/
final Version version = this.version == Version.LUCENE_4_3 ? Version.LUCENE_4_4 : this.version; // always use 4.4 or higher
if (matcher == null) {
return new NGramTokenizer(minGram, maxGram);
} else {
return new NGramTokenizer(minGram, maxGram) {
@Override
protected boolean isTokenChar(int chr) {
return matcher.isTokenChar(chr);
}
};
}
if (matcher == null) {
return new NGramTokenizer(minGram, maxGram);
} else {
return new Lucene43NGramTokenizer(minGram, maxGram);
return new NGramTokenizer(minGram, maxGram) {
@Override
protected boolean isTokenChar(int chr) {
return matcher.isTokenChar(chr);
}
};
}
}

View File

@ -183,7 +183,7 @@ class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler {
boolean isEnabled = getIORateLimitMBPerSec() != Double.POSITIVE_INFINITY;
if (config.isAutoThrottle() && isEnabled == false) {
enableAutoIOThrottle();
} else if (config.isAutoThrottle() == false && isEnabled){
} else if (config.isAutoThrottle() == false && isEnabled) {
disableAutoIOThrottle();
}
}

View File

@ -35,6 +35,7 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.search.join.BitSetProducer;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.elasticsearch.ExceptionsHelper;
@ -430,8 +431,8 @@ public abstract class Engine implements Closeable {
stats.addIndexWriterMaxMemoryInBytes(0);
}
/** How much heap Lucene's IndexWriter is using */
abstract public long indexWriterRAMBytesUsed();
/** How much heap is used that would be freed by a refresh. Note that this may throw {@link AlreadyClosedException}. */
abstract public long getIndexBufferRAMBytesUsed();
protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) {
ensureOpen();
@ -529,11 +530,17 @@ public abstract class Engine implements Closeable {
}
/**
* Refreshes the engine for new search operations to reflect the latest
* Synchronously refreshes the engine for new search operations to reflect the latest
* changes.
*/
public abstract void refresh(String source) throws EngineException;
/**
* Called when our engine is using too much heap and should move buffered indexed/deleted documents to disk.
*/
// NOTE: do NOT rename this to something containing flush or refresh!
public abstract void writeIndexingBuffer() throws EngineException;
/**
* Flushes the state of the engine including the transaction log, clearing memory.
*
@ -1142,4 +1149,14 @@ public abstract class Engine implements Closeable {
*/
void warm(Engine.Searcher searcher, boolean isTopLevelReader);
}
/**
* Request that this engine throttle incoming indexing requests to one thread. Must be matched by a later call to {@link deactivateThrottling}.
*/
public abstract void activateThrottling();
/**
* Reverses a previous {@link #activateThrottling} call.
*/
public abstract void deactivateThrottling();
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexSettings;
@ -49,9 +50,7 @@ public final class EngineConfig {
private final ShardId shardId;
private final TranslogRecoveryPerformer translogRecoveryPerformer;
private final IndexSettings indexSettings;
private volatile ByteSizeValue indexingBufferSize;
private volatile ByteSizeValue versionMapSize;
private volatile String versionMapSizeSetting;
private final ByteSizeValue indexingBufferSize;
private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis();
private volatile boolean enableGcDeletes = true;
private final TimeValue flushMergesAfter;
@ -82,22 +81,12 @@ public final class EngineConfig {
*/
public static final String INDEX_CODEC_SETTING = "index.codec";
/**
* The maximum size the version map should grow to before issuing a refresh. Can be an absolute value or a percentage of
* the current index memory buffer (defaults to 25%)
*/
public static final String INDEX_VERSION_MAP_SIZE = "index.version_map_size";
/** if set to true the engine will start even if the translog id in the commit point can not be found */
public static final String INDEX_FORCE_NEW_TRANSLOG = "index.engine.force_new_translog";
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
public static final String DEFAULT_VERSION_MAP_SIZE = "25%";
private static final String DEFAULT_CODEC_NAME = "default";
private TranslogConfig translogConfig;
private boolean create = false;
@ -124,11 +113,11 @@ public final class EngineConfig {
this.codecService = codecService;
this.eventListener = eventListener;
codecName = settings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
// We start up inactive and rely on IndexingMemoryController to give us our fair share once we start indexing:
indexingBufferSize = IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER;
// We give IndexWriter a "huge" (256 MB) buffer, so it won't flush on its own unless the ES indexing buffer is also huge and/or
// there are not too many shards allocated to this node. Instead, IndexingMemoryController periodically checks
// and refreshes the most heap-consuming shards when total indexing heap usage across all shards is too high:
indexingBufferSize = new ByteSizeValue(256, ByteSizeUnit.MB);
gcDeletesInMillis = settings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis();
versionMapSizeSetting = settings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize();
this.translogRecoveryPerformer = translogRecoveryPerformer;
this.forceNewTranslog = settings.getAsBoolean(INDEX_FORCE_NEW_TRANSLOG, false);
this.queryCache = queryCache;
@ -137,51 +126,11 @@ public final class EngineConfig {
this.flushMergesAfter = flushMergesAfter;
}
/** updates {@link #versionMapSize} based on current setting and {@link #indexingBufferSize} */
private void updateVersionMapSize() {
if (versionMapSizeSetting.endsWith("%")) {
double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1));
versionMapSize = new ByteSizeValue((long) ((double) indexingBufferSize.bytes() * (percent / 100)));
} else {
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting, INDEX_VERSION_MAP_SIZE);
}
}
/**
* Settings the version map size that should trigger a refresh. See {@link #INDEX_VERSION_MAP_SIZE} for details.
*/
public void setVersionMapSizeSetting(String versionMapSizeSetting) {
this.versionMapSizeSetting = versionMapSizeSetting;
updateVersionMapSize();
}
/**
* current setting for the version map size that should trigger a refresh. See {@link #INDEX_VERSION_MAP_SIZE} for details.
*/
public String getVersionMapSizeSetting() {
return versionMapSizeSetting;
}
/** if true the engine will start even if the translog id in the commit point can not be found */
public boolean forceNewTranslog() {
return forceNewTranslog;
}
/**
* returns the size of the version map that should trigger a refresh
*/
public ByteSizeValue getVersionMapSize() {
return versionMapSize;
}
/**
* Sets the indexing buffer
*/
public void setIndexingBufferSize(ByteSizeValue indexingBufferSize) {
this.indexingBufferSize = indexingBufferSize;
updateVersionMapSize();
}
/**
* Enables / disables gc deletes
*

View File

@ -55,6 +55,7 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.math.MathUtils;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
@ -118,6 +119,11 @@ public class InternalEngine extends Engine {
private final IndexThrottle throttle;
// How many callers are currently requesting index throttling. Currently there are only two situations where we do this: when merges
// are falling behind and when writing indexing buffer to disk is too slow. When this is 0, there is no throttling, else we throttling
// incoming indexing ops to a single thread:
private final AtomicInteger throttleRequestCount = new AtomicInteger();
public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogRecovery) throws EngineException {
super(engineConfig);
this.versionMap = new LiveVersionMap();
@ -306,15 +312,6 @@ public class InternalEngine extends Engine {
}
}
private void updateIndexWriterSettings() {
try {
final LiveIndexWriterConfig iwc = indexWriter.getConfig();
iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().mbFrac());
} catch (AlreadyClosedException ex) {
// ignore
}
}
@Override
public GetResult get(Get get, Function<String, Searcher> searcherFactory) throws EngineException {
try (ReleasableLock lock = readLock.acquire()) {
@ -359,13 +356,12 @@ public class InternalEngine extends Engine {
maybeFailEngine("index", t);
throw new IndexFailedEngineException(shardId, index.type(), index.id(), t);
}
checkVersionMapRefresh();
return created;
}
private boolean innerIndex(Index index) throws IOException {
synchronized (dirtyLock(index.uid())) {
lastWriteNanos = index.startTime();
lastWriteNanos = index.startTime();
final long currentVersion;
final boolean deleted;
VersionValue versionValue = versionMap.getUnderLock(index.uid().bytes());
@ -423,33 +419,6 @@ public class InternalEngine extends Engine {
}
}
/**
* Forces a refresh if the versionMap is using too much RAM
*/
private void checkVersionMapRefresh() {
if (versionMap.ramBytesUsedForRefresh() > config().getVersionMapSize().bytes() && versionMapRefreshPending.getAndSet(true) == false) {
try {
if (isClosed.get()) {
// no point...
return;
}
// Now refresh to clear versionMap:
engineConfig.getThreadPool().executor(ThreadPool.Names.REFRESH).execute(new Runnable() {
@Override
public void run() {
try {
refresh("version_table_full");
} catch (EngineClosedException ex) {
// ignore
}
}
});
} catch (EsRejectedExecutionException ex) {
// that is fine too.. we might be shutting down
}
}
}
@Override
public void delete(Delete delete) throws EngineException {
try (ReleasableLock lock = readLock.acquire()) {
@ -462,7 +431,6 @@ public class InternalEngine extends Engine {
}
maybePruneDeletedTombstones();
checkVersionMapRefresh();
}
private void maybePruneDeletedTombstones() {
@ -547,6 +515,43 @@ public class InternalEngine extends Engine {
mergeScheduler.refreshConfig();
}
@Override
public void writeIndexingBuffer() throws EngineException {
// we obtain a read lock here, since we don't want a flush to happen while we are writing
// since it flushes the index as well (though, in terms of concurrency, we are allowed to do it)
try (ReleasableLock lock = readLock.acquire()) {
ensureOpen();
// TODO: it's not great that we secretly tie searcher visibility to "freeing up heap" here... really we should keep two
// searcher managers, one for searching which is only refreshed by the schedule the user requested (refresh_interval, or invoking
// refresh API), and another for version map interactions. See #15768.
final long versionMapBytes = versionMap.ramBytesUsedForRefresh();
final long indexingBufferBytes = indexWriter.ramBytesUsed();
final boolean useRefresh = versionMapRefreshPending.get() || (indexingBufferBytes/4 < versionMapBytes);
if (useRefresh) {
// The version map is using > 25% of the indexing buffer, so we do a refresh so the version map also clears
logger.debug("use refresh to write indexing buffer (heap size=[{}]), to also clear version map (heap size=[{}])",
new ByteSizeValue(indexingBufferBytes), new ByteSizeValue(versionMapBytes));
refresh("write indexing buffer");
} else {
// Most of our heap is used by the indexing buffer, so we do a cheaper (just writes segments, doesn't open a new searcher) IW.flush:
logger.debug("use IndexWriter.flush to write indexing buffer (heap size=[{}]) since version map is small (heap size=[{}])",
new ByteSizeValue(indexingBufferBytes), new ByteSizeValue(versionMapBytes));
indexWriter.flush();
}
} catch (AlreadyClosedException e) {
ensureOpen();
maybeFailEngine("writeIndexingBuffer", e);
} catch (EngineClosedException e) {
throw e;
} catch (Throwable t) {
failEngine("writeIndexingBuffer failed", t);
throw new RefreshFailedEngineException(shardId, t);
}
}
@Override
public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException {
// best effort attempt before we acquire locks
@ -821,8 +826,8 @@ public class InternalEngine extends Engine {
}
@Override
public long indexWriterRAMBytesUsed() {
return indexWriter.ramBytesUsed();
public long getIndexBufferRAMBytesUsed() {
return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh();
}
@Override
@ -1044,12 +1049,22 @@ public class InternalEngine extends Engine {
}
}
@Override
public void activateThrottling() {
throttle.activate();
int count = throttleRequestCount.incrementAndGet();
assert count >= 1: "invalid post-increment throttleRequestCount=" + count;
if (count == 1) {
throttle.activate();
}
}
@Override
public void deactivateThrottling() {
throttle.deactivate();
int count = throttleRequestCount.decrementAndGet();
assert count >= 0: "invalid post-decrement throttleRequestCount=" + count;
if (count == 0) {
throttle.deactivate();
}
}
public long getIndexThrottleTimeInMillis() {
@ -1162,9 +1177,6 @@ public class InternalEngine extends Engine {
public void onSettingsChanged() {
mergeScheduler.refreshConfig();
updateIndexWriterSettings();
// config().getVersionMapSize() may have changed:
checkVersionMapRefresh();
// config().isEnableGcDeletes() or config.getGcDeletesInMillis() may have changed:
maybePruneDeletedTombstones();
}

View File

@ -227,8 +227,24 @@ public class ShadowEngine extends Engine {
}
@Override
public long indexWriterRAMBytesUsed() {
// No IndexWriter
public long getIndexBufferRAMBytesUsed() {
// No IndexWriter nor version map
throw new UnsupportedOperationException("ShadowEngine has no IndexWriter");
}
@Override
public void writeIndexingBuffer() {
// No indexing buffer
throw new UnsupportedOperationException("ShadowEngine has no IndexWriter");
}
@Override
public void activateThrottling() {
throw new UnsupportedOperationException("ShadowEngine has no IndexWriter");
}
@Override
public void deactivateThrottling() {
throw new UnsupportedOperationException("ShadowEngine has no IndexWriter");
}
}

View File

@ -604,14 +604,14 @@ public abstract class QueryBuilders {
* Facilitates creating template query requests using an inline script
*/
public static TemplateQueryBuilder templateQuery(String template, Map<String, Object> vars) {
return new TemplateQueryBuilder(template, vars);
return new TemplateQueryBuilder(new Template(template, ScriptService.ScriptType.INLINE, null, null, vars));
}
/**
* Facilitates creating template query requests
*/
public static TemplateQueryBuilder templateQuery(String template, ScriptService.ScriptType templateType, Map<String, Object> vars) {
return new TemplateQueryBuilder(template, templateType, vars);
return new TemplateQueryBuilder(new Template(template, templateType, null, null, vars));
}
/**

View File

@ -70,7 +70,6 @@ public interface IndexEventListener {
*/
default void afterIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) {}
/**
* Called after a shard's {@link org.elasticsearch.index.shard.IndexShardState} changes.
* The order of concurrent events is preserved. The execution must be lightweight.
@ -89,13 +88,6 @@ public interface IndexEventListener {
*/
default void onShardInactive(IndexShard indexShard) {}
/**
* Called when a shard is marked as active ie. was previously inactive and is now active again.
*
* @param indexShard The shard that was marked active
*/
default void onShardActive(IndexShard indexShard) {}
/**
* Called before the index gets created. Note that this is also called
* when the index is created on data nodes

View File

@ -97,8 +97,8 @@ import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.search.stats.ShardSearchStats;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.snapshots.IndexShardRepository;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.Store.MetadataSnapshot;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.index.suggest.stats.ShardSuggestMetric;
@ -133,9 +133,9 @@ import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
public class IndexShard extends AbstractIndexShardComponent {
private final ThreadPool threadPool;
@ -167,6 +167,12 @@ public class IndexShard extends AbstractIndexShardComponent {
private final IndexEventListener indexEventListener;
private final IndexSettings idxSettings;
private final NodeServicesProvider provider;
/** How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this
* across all shards to decide if throttling is necessary because moving bytes to disk is falling behind vs incoming documents
* being indexed/deleted. */
private final AtomicLong writingBytes = new AtomicLong();
private TimeValue refreshInterval;
private volatile ScheduledFuture<?> refreshScheduledFuture;
@ -194,9 +200,7 @@ public class IndexShard extends AbstractIndexShardComponent {
*/
public static final String INDEX_FLUSH_ON_CLOSE = "index.flush_on_close";
public static final String INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE = "index.translog.flush_threshold_size";
/** If we see no indexing operations after this much time for a given shard, we consider that shard inactive (default: 5 minutes). */
public static final String INDEX_SHARD_INACTIVE_TIME_SETTING = "index.shard.inactive_time";
private static final String INDICES_INACTIVE_TIME_SETTING = "indices.memory.shard_inactive_time";
public static final String INDEX_REFRESH_INTERVAL = "index.refresh_interval";
private final ShardPath path;
@ -205,7 +209,6 @@ public class IndexShard extends AbstractIndexShardComponent {
private final EnumSet<IndexShardState> readAllowedStates = EnumSet.of(IndexShardState.STARTED, IndexShardState.RELOCATED, IndexShardState.POST_RECOVERY);
private final IndexSearcherWrapper searcherWrapper;
private final TimeValue inactiveTime;
/**
* True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link
@ -219,7 +222,6 @@ public class IndexShard extends AbstractIndexShardComponent {
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider, IndexingOperationListener... listeners) {
super(shardId, indexSettings);
final Settings settings = indexSettings.getSettings();
this.inactiveTime = settings.getAsTime(INDEX_SHARD_INACTIVE_TIME_SETTING, settings.getAsTime(INDICES_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5)));
this.idxSettings = indexSettings;
this.codecService = new CodecService(mapperService, logger);
this.warmer = provider.getWarmer();
@ -272,8 +274,6 @@ public class IndexShard extends AbstractIndexShardComponent {
this.provider = provider;
this.searcherWrapper = indexSearcherWrapper;
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, newQueryShardContext());
// We start up inactive
active.set(false);
}
public Store store() {
@ -488,7 +488,7 @@ public class IndexShard extends AbstractIndexShardComponent {
*/
public boolean index(Engine.Index index) {
ensureWriteAllowed(index);
markLastWrite();
active.set(true);
index = indexingOperationListeners.preIndex(index);
final boolean created;
try {
@ -506,7 +506,9 @@ public class IndexShard extends AbstractIndexShardComponent {
indexingOperationListeners.postIndex(index, ex);
throw ex;
}
indexingOperationListeners.postIndex(index);
return created;
}
@ -528,10 +530,9 @@ public class IndexShard extends AbstractIndexShardComponent {
return new Engine.Delete(type, id, uid, version, versionType, origin, startTime, false);
}
public void delete(Engine.Delete delete) {
ensureWriteAllowed(delete);
markLastWrite();
active.set(true);
delete = indexingOperationListeners.preDelete(delete);
try {
if (logger.isTraceEnabled()) {
@ -548,6 +549,7 @@ public class IndexShard extends AbstractIndexShardComponent {
indexingOperationListeners.postDelete(delete, ex);
throw ex;
}
indexingOperationListeners.postDelete(delete);
}
@ -556,14 +558,32 @@ public class IndexShard extends AbstractIndexShardComponent {
return getEngine().get(get, this::acquireSearcher);
}
/** Writes all indexing changes to disk and opens a new searcher reflecting all changes. This can throw {@link EngineClosedException}. */
public void refresh(String source) {
verifyNotClosed();
if (logger.isTraceEnabled()) {
logger.trace("refresh with source: {}", source);
if (canIndex()) {
long bytes = getEngine().getIndexBufferRAMBytesUsed();
writingBytes.addAndGet(bytes);
try {
logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(bytes));
long time = System.nanoTime();
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
} finally {
logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId());
writingBytes.addAndGet(-bytes);
}
} else {
logger.debug("refresh with source [{}]", source);
long time = System.nanoTime();
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
}
long time = System.nanoTime();
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
}
/** Returns how many bytes we are currently moving from heap to disk */
public long getWritingBytes() {
return writingBytes.get();
}
public RefreshStats refreshStats() {
@ -954,13 +974,6 @@ public class IndexShard extends AbstractIndexShardComponent {
}
}
/** Records timestamp of the last write operation, possibly switching {@code active} to true if we were inactive. */
private void markLastWrite() {
if (active.getAndSet(true) == false) {
indexEventListener.onShardActive(this);
}
}
private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardStateException {
Engine.Operation.Origin origin = op.origin();
IndexShardState state = this.state; // one time volatile read
@ -1018,85 +1031,34 @@ public class IndexShard extends AbstractIndexShardComponent {
}
}
public static final String INDEX_REFRESH_INTERVAL = "index.refresh_interval";
/** Returns number of heap bytes used by the indexing buffer for this shard, or 0 if the shard is closed */
public long getIndexBufferRAMBytesUsed() {
Engine engine = getEngineOrNull();
if (engine == null) {
return 0;
}
try {
return engine.getIndexBufferRAMBytesUsed();
} catch (AlreadyClosedException ex) {
return 0;
}
}
public void addShardFailureCallback(Callback<ShardFailure> onShardFailure) {
this.shardEventListener.delegates.add(onShardFailure);
}
/**
* Change the indexing and translog buffer sizes. If {@code IndexWriter} is currently using more than
* the new buffering indexing size then we do a refresh to free up the heap.
*/
public void updateBufferSize(ByteSizeValue shardIndexingBufferSize) {
final EngineConfig config = engineConfig;
final ByteSizeValue preValue = config.getIndexingBufferSize();
config.setIndexingBufferSize(shardIndexingBufferSize);
Engine engine = getEngineOrNull();
if (engine == null) {
logger.debug("updateBufferSize: engine is closed; skipping");
return;
}
// update engine if it is already started.
if (preValue.bytes() != shardIndexingBufferSize.bytes()) {
// so we push changes these changes down to IndexWriter:
engine.onSettingsChanged();
long iwBytesUsed = engine.indexWriterRAMBytesUsed();
String message = LoggerMessageFormat.format("updating index_buffer_size from [{}] to [{}]; IndexWriter now using [{}] bytes",
preValue, shardIndexingBufferSize, iwBytesUsed);
if (iwBytesUsed > shardIndexingBufferSize.bytes()) {
// our allowed buffer was changed to less than we are currently using; we ask IW to refresh
// so it clears its buffers (otherwise it won't clear until the next indexing/delete op)
logger.debug(message + "; now refresh to clear IndexWriter memory");
// TODO: should IW have an API to move segments to disk, but not refresh? Its flush method is protected...
try {
refresh("update index buffer");
} catch (Throwable e) {
logger.warn("failed to refresh after decreasing index buffer", e);
}
} else {
logger.debug(message);
}
}
}
/**
* Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last
* indexing operation, and become inactive (reducing indexing and translog buffers to tiny values) if so. This returns true
* if the shard is inactive.
*/
public boolean checkIdle() {
return checkIdle(inactiveTime.nanos());
}
final boolean checkIdle(long inactiveTimeNS) { // pkg private for testing
/** Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last
* indexing operation, and notify listeners that we are now inactive so e.g. sync'd flush can happen. */
public void checkIdle(long inactiveTimeNS) {
Engine engineOrNull = getEngineOrNull();
if (engineOrNull != null && System.nanoTime() - engineOrNull.getLastWriteNanos() >= inactiveTimeNS) {
boolean wasActive = active.getAndSet(false);
if (wasActive) {
updateBufferSize(IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER);
logger.debug("marking shard as inactive (inactive_time=[{}]) indexing wise", inactiveTime);
logger.debug("shard is now inactive");
indexEventListener.onShardInactive(this);
}
}
return active.get() == false;
}
/**
* Returns {@code true} if this shard is active (has seen indexing ops in the last {@link
* IndexShard#INDEX_SHARD_INACTIVE_TIME_SETTING} (default 5 minutes), else {@code false}.
*/
public boolean getActive() {
return active.get();
}
public final boolean isFlushOnClose() {
@ -1194,11 +1156,6 @@ public class IndexShard extends AbstractIndexShardComponent {
change = true;
}
final String versionMapSize = settings.get(EngineConfig.INDEX_VERSION_MAP_SIZE, config.getVersionMapSizeSetting());
if (config.getVersionMapSizeSetting().equals(versionMapSize) == false) {
config.setVersionMapSizeSetting(versionMapSize);
}
final int maxThreadCount = settings.getAsInt(MergeSchedulerConfig.MAX_THREAD_COUNT, mergeSchedulerConfig.getMaxThreadCount());
if (maxThreadCount != mergeSchedulerConfig.getMaxThreadCount()) {
logger.info("updating [{}] from [{}] to [{}]", MergeSchedulerConfig.MAX_THREAD_COUNT, mergeSchedulerConfig.getMaxMergeCount(), maxThreadCount);
@ -1253,8 +1210,70 @@ public class IndexShard extends AbstractIndexShardComponent {
return indexEventListener;
}
public TimeValue getInactiveTime() {
return inactiveTime;
public void activateThrottling() {
try {
getEngine().activateThrottling();
} catch (EngineClosedException ex) {
// ignore
}
}
public void deactivateThrottling() {
try {
getEngine().deactivateThrottling();
} catch (EngineClosedException ex) {
// ignore
}
}
private void handleRefreshException(Exception e) {
if (e instanceof EngineClosedException) {
// ignore
} else if (e instanceof RefreshFailedEngineException) {
RefreshFailedEngineException rfee = (RefreshFailedEngineException) e;
if (rfee.getCause() instanceof InterruptedException) {
// ignore, we are being shutdown
} else if (rfee.getCause() instanceof ClosedByInterruptException) {
// ignore, we are being shutdown
} else if (rfee.getCause() instanceof ThreadInterruptedException) {
// ignore, we are being shutdown
} else {
if (state != IndexShardState.CLOSED) {
logger.warn("Failed to perform engine refresh", e);
}
}
} else {
if (state != IndexShardState.CLOSED) {
logger.warn("Failed to perform engine refresh", e);
}
}
}
/**
* Called when our shard is using too much heap and should move buffered indexed/deleted documents to disk.
*/
public void writeIndexingBuffer() {
if (canIndex() == false) {
throw new UnsupportedOperationException();
}
try {
Engine engine = getEngine();
long bytes = engine.getIndexBufferRAMBytesUsed();
// NOTE: this can be an overestimate by up to 20%, if engine uses IW.flush not refresh, because version map
// memory is low enough, but this is fine because after the writes finish, IMC will poll again and see that
// there's still up to the 20% being used and continue writing if necessary:
logger.debug("add [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId());
writingBytes.addAndGet(bytes);
try {
engine.writeIndexingBuffer();
} finally {
writingBytes.addAndGet(-bytes);
logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId());
}
} catch (Exception e) {
handleRefreshException(e);
};
}
/**
@ -1265,7 +1284,7 @@ public class IndexShard extends AbstractIndexShardComponent {
internalIndexingStats.noopUpdate(type);
}
class EngineRefresher implements Runnable {
final class EngineRefresher implements Runnable {
@Override
public void run() {
// we check before if a refresh is needed, if not, we reschedule, otherwise, we fork, refresh, and then reschedule
@ -1277,27 +1296,13 @@ public class IndexShard extends AbstractIndexShardComponent {
@Override
public void run() {
try {
// TODO: now that we use refresh to clear the indexing buffer, we should check here if we did that "recently" and
// reschedule if so...
if (getEngine().refreshNeeded()) {
refresh("schedule");
}
} catch (EngineClosedException e) {
// we are being closed, ignore
} catch (RefreshFailedEngineException e) {
if (e.getCause() instanceof InterruptedException) {
// ignore, we are being shutdown
} else if (e.getCause() instanceof ClosedByInterruptException) {
// ignore, we are being shutdown
} else if (e.getCause() instanceof ThreadInterruptedException) {
// ignore, we are being shutdown
} else {
if (state != IndexShardState.CLOSED) {
logger.warn("Failed to perform scheduled engine refresh", e);
}
}
} catch (Exception e) {
if (state != IndexShardState.CLOSED) {
logger.warn("Failed to perform scheduled engine refresh", e);
}
handleRefreshException(e);
}
reschedule();
@ -1493,7 +1498,8 @@ public class IndexShard extends AbstractIndexShardComponent {
final Engine.Warmer engineWarmer = (searcher, toLevel) -> warmer.warm(searcher, this, idxSettings, toLevel);
return new EngineConfig(shardId,
threadPool, indexSettings, engineWarmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig,
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, inactiveTime);
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig,
idxSettings.getSettings().getAsTime(IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, IndexingMemoryController.SHARD_DEFAULT_INACTIVE_TIME));
}
private static class IndexShardOperationCounter extends AbstractRefCounted {

View File

@ -118,6 +118,7 @@ import static java.util.Collections.unmodifiableMap;
* </pre>
*/
public class Store extends AbstractIndexShardComponent implements Closeable, RefCounted {
private static final Version FIRST_LUCENE_CHECKSUM_VERSION = Version.LUCENE_4_8_0;
static final String CODEC = "store";
static final int VERSION_WRITE_THROWABLE= 2; // we write throwable since 2.0
@ -466,7 +467,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
output = new LegacyVerification.LengthVerifyingIndexOutput(output, metadata.length());
} else {
assert metadata.writtenBy() != null;
assert metadata.writtenBy().onOrAfter(Version.LUCENE_4_8);
assert metadata.writtenBy().onOrAfter(FIRST_LUCENE_CHECKSUM_VERSION);
output = new LuceneVerifyingIndexOutput(metadata, output);
}
success = true;
@ -490,7 +491,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
return directory().openInput(filename, context);
}
assert metadata.writtenBy() != null;
assert metadata.writtenBy().onOrAfter(Version.LUCENE_4_8_0);
assert metadata.writtenBy().onOrAfter(FIRST_LUCENE_CHECKSUM_VERSION);
return new VerifyingIndexInput(directory().openInput(filename, context));
}
@ -518,7 +519,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
if (input.length() != md.length()) { // first check the length no matter how old this file is
throw new CorruptIndexException("expected length=" + md.length() + " != actual length: " + input.length() + " : file truncated?", input);
}
if (md.writtenBy() != null && md.writtenBy().onOrAfter(Version.LUCENE_4_8_0)) {
if (md.writtenBy() != null && md.writtenBy().onOrAfter(FIRST_LUCENE_CHECKSUM_VERSION)) {
// throw exception if the file is corrupt
String checksum = Store.digestToString(CodecUtil.checksumEntireFile(input));
// throw exception if metadata is inconsistent
@ -766,7 +767,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
*/
public final static class MetadataSnapshot implements Iterable<StoreFileMetaData>, Writeable<MetadataSnapshot> {
private static final ESLogger logger = Loggers.getLogger(MetadataSnapshot.class);
private static final Version FIRST_LUCENE_CHECKSUM_VERSION = Version.LUCENE_4_8;
private final Map<String, StoreFileMetaData> metadata;
@ -843,6 +843,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory);
numDocs = Lucene.getNumDocs(segmentCommitInfos);
commitUserDataBuilder.putAll(segmentCommitInfos.getUserData());
@SuppressWarnings("deprecation")
Version maxVersion = Version.LUCENE_4_0; // we don't know which version was used to write so we take the max version.
for (SegmentCommitInfo info : segmentCommitInfos) {
final Version version = info.info.getVersion();
@ -907,6 +908,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
* @param directory the directory to read checksums from
* @return a map of file checksums and the checksum file version
*/
@SuppressWarnings("deprecation") // Legacy checksum needs legacy methods
static Tuple<Map<String, String>, Long> readLegacyChecksums(Directory directory) throws IOException {
synchronized (directory) {
long lastFound = -1;
@ -922,10 +924,10 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
if (lastFound > -1) {
try (IndexInput indexInput = directory.openInput(CHECKSUMS_PREFIX + lastFound, IOContext.READONCE)) {
indexInput.readInt(); // version
return new Tuple(indexInput.readStringStringMap(), lastFound);
return new Tuple<>(indexInput.readStringStringMap(), lastFound);
}
}
return new Tuple(new HashMap<>(), -1l);
return new Tuple<>(new HashMap<>(), -1l);
}
}
@ -1243,6 +1245,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
}
}
@SuppressWarnings("deprecation") // Legacy checksum uses legacy methods
synchronized void writeChecksums(Directory directory, Map<String, String> checksums, long lastVersion) throws IOException {
// Make sure if clock goes backwards we still move version forwards:
long nextVersion = Math.max(lastVersion+1, System.currentTimeMillis());

View File

@ -92,7 +92,7 @@ public class TranslogWriter extends TranslogReader {
writeCheckpoint(headerLength, 0, file.getParent(), fileGeneration, StandardOpenOption.WRITE);
final TranslogWriter writer = new TranslogWriter(shardId, fileGeneration, new ChannelReference(file, fileGeneration, channel, onClose), bufferSize);
return writer;
} catch (Throwable throwable){
} catch (Throwable throwable) {
// if we fail to bake the file-generation into the checkpoint we stick with the file and once we recover and that
// file exists we remove it. We only apply this logic to the checkpoint.generation+1 any other file with a higher generation is an error condition
IOUtils.closeWhileHandlingException(channel);

View File

@ -24,23 +24,32 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineClosedException;
import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.IndexingOperationListener;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.PriorityQueue;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
public class IndexingMemoryController extends AbstractComponent implements IndexEventListener, Closeable {
public class IndexingMemoryController extends AbstractComponent implements IndexingOperationListener, Closeable {
/** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */
public static final String INDEX_BUFFER_SIZE_SETTING = "indices.memory.index_buffer_size";
@ -51,34 +60,30 @@ public class IndexingMemoryController extends AbstractComponent implements Index
/** Only applies when <code>indices.memory.index_buffer_size</code> is a %, to set a ceiling on the actual size in bytes (default: not set). */
public static final String MAX_INDEX_BUFFER_SIZE_SETTING = "indices.memory.max_index_buffer_size";
/** Sets a floor on the per-shard index buffer size (default: 4 MB). */
public static final String MIN_SHARD_INDEX_BUFFER_SIZE_SETTING = "indices.memory.min_shard_index_buffer_size";
/** If we see no indexing operations after this much time for a given shard, we consider that shard inactive (default: 5 minutes). */
public static final String SHARD_INACTIVE_TIME_SETTING = "indices.memory.shard_inactive_time";
/** Sets a ceiling on the per-shard index buffer size (default: 512 MB). */
public static final String MAX_SHARD_INDEX_BUFFER_SIZE_SETTING = "indices.memory.max_shard_index_buffer_size";
/** Default value (5 minutes) for indices.memory.shard_inactive_time */
public static final TimeValue SHARD_DEFAULT_INACTIVE_TIME = TimeValue.timeValueMinutes(5);
/** Sets a floor on the per-shard translog buffer size (default: 2 KB). */
public static final String MIN_SHARD_TRANSLOG_BUFFER_SIZE_SETTING = "indices.memory.min_shard_translog_buffer_size";
/** How frequently we check indexing memory usage (default: 5 seconds). */
public static final String SHARD_MEMORY_INTERVAL_TIME_SETTING = "indices.memory.interval";
/** Sets a ceiling on the per-shard translog buffer size (default: 64 KB). */
public static final String MAX_SHARD_TRANSLOG_BUFFER_SIZE_SETTING = "indices.memory.max_shard_translog_buffer_size";
/** How frequently we check shards to find inactive ones (default: 30 seconds). */
public static final String SHARD_INACTIVE_INTERVAL_TIME_SETTING = "indices.memory.interval";
/** Once a shard becomes inactive, we reduce the {@code IndexWriter} buffer to this value (500 KB) to let active shards use the heap instead. */
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
private final ThreadPool threadPool;
private final IndicesService indicesService;
private final ByteSizeValue indexingBuffer;
private final ByteSizeValue minShardIndexBufferSize;
private final ByteSizeValue maxShardIndexBufferSize;
private final TimeValue inactiveTime;
private final TimeValue interval;
/** Contains shards currently being throttled because we can't write segments quickly enough */
private final Set<IndexShard> throttled = new HashSet<>();
private final ScheduledFuture scheduler;
private static final EnumSet<IndexShardState> CAN_UPDATE_INDEX_BUFFER_STATES = EnumSet.of(
private static final EnumSet<IndexShardState> CAN_WRITE_INDEX_BUFFER_STATES = EnumSet.of(
IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED, IndexShardState.RELOCATED);
private final ShardsIndicesStatusChecker statusChecker;
@ -110,21 +115,21 @@ public class IndexingMemoryController extends AbstractComponent implements Index
indexingBuffer = ByteSizeValue.parseBytesSizeValue(indexingBufferSetting, INDEX_BUFFER_SIZE_SETTING);
}
this.indexingBuffer = indexingBuffer;
this.minShardIndexBufferSize = this.settings.getAsBytesSize(MIN_SHARD_INDEX_BUFFER_SIZE_SETTING, new ByteSizeValue(4, ByteSizeUnit.MB));
// LUCENE MONITOR: Based on this thread, currently (based on Mike), having a large buffer does not make a lot of sense: https://issues.apache.org/jira/browse/LUCENE-2324?focusedCommentId=13005155&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13005155
this.maxShardIndexBufferSize = this.settings.getAsBytesSize(MAX_SHARD_INDEX_BUFFER_SIZE_SETTING, new ByteSizeValue(512, ByteSizeUnit.MB));
// we need to have this relatively small to move a shard from inactive to active fast (enough)
this.interval = this.settings.getAsTime(SHARD_INACTIVE_INTERVAL_TIME_SETTING, TimeValue.timeValueSeconds(30));
this.inactiveTime = this.settings.getAsTime(SHARD_INACTIVE_TIME_SETTING, SHARD_DEFAULT_INACTIVE_TIME);
// we need to have this relatively small to free up heap quickly enough
this.interval = this.settings.getAsTime(SHARD_MEMORY_INTERVAL_TIME_SETTING, TimeValue.timeValueSeconds(5));
this.statusChecker = new ShardsIndicesStatusChecker();
logger.debug("using indexing buffer size [{}], with {} [{}], {} [{}], {} [{}]",
this.indexingBuffer,
MIN_SHARD_INDEX_BUFFER_SIZE_SETTING, this.minShardIndexBufferSize,
MAX_SHARD_INDEX_BUFFER_SIZE_SETTING, this.maxShardIndexBufferSize,
SHARD_INACTIVE_INTERVAL_TIME_SETTING, this.interval);
logger.debug("using indexing buffer size [{}] with {} [{}], {} [{}]",
this.indexingBuffer,
SHARD_INACTIVE_TIME_SETTING, this.inactiveTime,
SHARD_MEMORY_INTERVAL_TIME_SETTING, this.interval);
this.scheduler = scheduleTask(threadPool);
// Need to save this so we can later launch async "write indexing buffer to disk" on shards:
this.threadPool = threadPool;
}
protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
@ -150,7 +155,8 @@ public class IndexingMemoryController extends AbstractComponent implements Index
for (IndexService indexService : indicesService) {
for (IndexShard shard : indexService) {
if (shardAvailable(shard)) {
// shadow replica doesn't have an indexing buffer
if (shard.canIndex() && CAN_WRITE_INDEX_BUFFER_STATES.contains(shard.state())) {
availableShards.add(shard);
}
}
@ -158,85 +164,220 @@ public class IndexingMemoryController extends AbstractComponent implements Index
return availableShards;
}
/** returns true if shard exists and is availabe for updates */
protected boolean shardAvailable(IndexShard shard) {
// shadow replica doesn't have an indexing buffer
return shard.canIndex() && CAN_UPDATE_INDEX_BUFFER_STATES.contains(shard.state());
/** returns how much heap this shard is using for its indexing buffer */
protected long getIndexBufferRAMBytesUsed(IndexShard shard) {
return shard.getIndexBufferRAMBytesUsed();
}
/** set new indexing and translog buffers on this shard. this may cause the shard to refresh to free up heap. */
protected void updateShardBuffers(IndexShard shard, ByteSizeValue shardIndexingBufferSize) {
try {
shard.updateBufferSize(shardIndexingBufferSize);
} catch (EngineClosedException | FlushNotAllowedEngineException e) {
// ignore
} catch (Exception e) {
logger.warn("failed to set shard {} index buffer to [{}]", e, shard.shardId(), shardIndexingBufferSize);
}
/** returns how many bytes this shard is currently writing to disk */
protected long getShardWritingBytes(IndexShard shard) {
return shard.getWritingBytes();
}
/** check if any shards active status changed, now. */
/** ask this shard to refresh, in the background, to free up heap */
protected void writeIndexingBufferAsync(IndexShard shard) {
threadPool.executor(ThreadPool.Names.REFRESH).execute(new AbstractRunnable() {
@Override
public void doRun() {
shard.writeIndexingBuffer();
}
@Override
public void onFailure(Throwable t) {
logger.warn("failed to write indexing buffer for shard [{}]; ignoring", t, shard.shardId());
}
});
}
/** force checker to run now */
void forceCheck() {
statusChecker.run();
}
class ShardsIndicesStatusChecker implements Runnable {
@Override
public synchronized void run() {
List<IndexShard> availableShards = availableShards();
List<IndexShard> activeShards = new ArrayList<>();
for (IndexShard shard : availableShards) {
if (!checkIdle(shard)) {
activeShards.add(shard);
}
}
int activeShardCount = activeShards.size();
// TODO: we could be smarter here by taking into account how RAM the IndexWriter on each shard
// is actually using (using IW.ramBytesUsed), so that small indices (e.g. Marvel) would not
// get the same indexing buffer as large indices. But it quickly gets tricky...
if (activeShardCount == 0) {
return;
}
ByteSizeValue shardIndexingBufferSize = new ByteSizeValue(indexingBuffer.bytes() / activeShardCount);
if (shardIndexingBufferSize.bytes() < minShardIndexBufferSize.bytes()) {
shardIndexingBufferSize = minShardIndexBufferSize;
}
if (shardIndexingBufferSize.bytes() > maxShardIndexBufferSize.bytes()) {
shardIndexingBufferSize = maxShardIndexBufferSize;
}
logger.debug("recalculating shard indexing buffer, total is [{}] with [{}] active shards, each shard set to indexing=[{}]", indexingBuffer, activeShardCount, shardIndexingBufferSize);
for (IndexShard shard : activeShards) {
updateShardBuffers(shard, shardIndexingBufferSize);
}
}
/** called by IndexShard to record that this many bytes were written to translog */
public void bytesWritten(int bytes) {
statusChecker.bytesWritten(bytes);
}
protected long currentTimeInNanos() {
return System.nanoTime();
/** Asks this shard to throttle indexing to one thread */
protected void activateThrottling(IndexShard shard) {
shard.activateThrottling();
}
/**
* ask this shard to check now whether it is inactive, and reduces its indexing and translog buffers if so.
* return false if the shard is not idle, otherwise true
*/
protected boolean checkIdle(IndexShard shard) {
try {
return shard.checkIdle();
} catch (EngineClosedException | FlushNotAllowedEngineException e) {
logger.trace("ignore [{}] while marking shard {} as inactive", e.getClass().getSimpleName(), shard.shardId());
return true;
}
/** Asks this shard to stop throttling indexing to one thread */
protected void deactivateThrottling(IndexShard shard) {
shard.deactivateThrottling();
}
@Override
public void onShardActive(IndexShard indexShard) {
// At least one shard used to be inactive ie. a new write operation just showed up.
// We try to fix the shards indexing buffer immediately. We could do this async instead, but cost should
// be low, and it's rare this happens.
forceCheck();
public void postIndex(Engine.Index index) {
bytesWritten(index.getTranslogLocation().size);
}
@Override
public void postDelete(Engine.Delete delete) {
bytesWritten(delete.getTranslogLocation().size);
}
private static final class ShardAndBytesUsed implements Comparable<ShardAndBytesUsed> {
final long bytesUsed;
final IndexShard shard;
public ShardAndBytesUsed(long bytesUsed, IndexShard shard) {
this.bytesUsed = bytesUsed;
this.shard = shard;
}
@Override
public int compareTo(ShardAndBytesUsed other) {
// Sort larger shards first:
return Long.compare(other.bytesUsed, bytesUsed);
}
}
/** not static because we need access to many fields/methods from our containing class (IMC): */
final class ShardsIndicesStatusChecker implements Runnable {
final AtomicLong bytesWrittenSinceCheck = new AtomicLong();
final ReentrantLock runLock = new ReentrantLock();
/** Shard calls this on each indexing/delete op */
public void bytesWritten(int bytes) {
long totalBytes = bytesWrittenSinceCheck.addAndGet(bytes);
while (totalBytes > indexingBuffer.bytes()/30) {
if (runLock.tryLock()) {
try {
bytesWrittenSinceCheck.addAndGet(-totalBytes);
// NOTE: this is only an approximate check, because bytes written is to the translog, vs indexing memory buffer which is
// typically smaller but can be larger in extreme cases (many unique terms). This logic is here only as a safety against
// thread starvation or too infrequent checking, to ensure we are still checking periodically, in proportion to bytes
// processed by indexing:
runUnlocked();
} finally {
runLock.unlock();
}
} else {
break;
}
}
}
@Override
public void run() {
runLock.lock();
try {
runUnlocked();
} finally {
runLock.unlock();
}
}
private void runUnlocked() {
// NOTE: even if we hit an errant exc here, our ThreadPool.scheduledWithFixedDelay will log the exception and re-invoke us
// again, on schedule
// First pass to sum up how much heap all shards' indexing buffers are using now, and how many bytes they are currently moving
// to disk:
long totalBytesUsed = 0;
long totalBytesWriting = 0;
for (IndexShard shard : availableShards()) {
// Give shard a chance to transition to inactive so sync'd flush can happen:
checkIdle(shard, inactiveTime.nanos());
// How many bytes this shard is currently (async'd) moving from heap to disk:
long shardWritingBytes = getShardWritingBytes(shard);
// How many heap bytes this shard is currently using
long shardBytesUsed = getIndexBufferRAMBytesUsed(shard);
shardBytesUsed -= shardWritingBytes;
totalBytesWriting += shardWritingBytes;
// If the refresh completed just after we pulled shardWritingBytes and before we pulled shardBytesUsed, then we could
// have a negative value here. So we just skip this shard since that means it's now using very little heap:
if (shardBytesUsed < 0) {
continue;
}
totalBytesUsed += shardBytesUsed;
}
if (logger.isTraceEnabled()) {
logger.trace("total indexing heap bytes used [{}] vs {} [{}], currently writing bytes [{}]",
new ByteSizeValue(totalBytesUsed), INDEX_BUFFER_SIZE_SETTING, indexingBuffer, new ByteSizeValue(totalBytesWriting));
}
// If we are using more than 50% of our budget across both indexing buffer and bytes we are still moving to disk, then we now
// throttle the top shards to send back-pressure to ongoing indexing:
boolean doThrottle = (totalBytesWriting + totalBytesUsed) > 1.5 * indexingBuffer.bytes();
if (totalBytesUsed > indexingBuffer.bytes()) {
// OK we are now over-budget; fill the priority queue and ask largest shard(s) to refresh:
PriorityQueue<ShardAndBytesUsed> queue = new PriorityQueue<>();
for (IndexShard shard : availableShards()) {
// How many bytes this shard is currently (async'd) moving from heap to disk:
long shardWritingBytes = getShardWritingBytes(shard);
// How many heap bytes this shard is currently using
long shardBytesUsed = getIndexBufferRAMBytesUsed(shard);
// Only count up bytes not already being refreshed:
shardBytesUsed -= shardWritingBytes;
// If the refresh completed just after we pulled shardWritingBytes and before we pulled shardBytesUsed, then we could
// have a negative value here. So we just skip this shard since that means it's now using very little heap:
if (shardBytesUsed < 0) {
continue;
}
if (shardBytesUsed > 0) {
if (logger.isTraceEnabled()) {
if (shardWritingBytes != 0) {
logger.trace("shard [{}] is using [{}] heap, writing [{}] heap", shard.shardId(), shardBytesUsed, shardWritingBytes);
} else {
logger.trace("shard [{}] is using [{}] heap, not writing any bytes", shard.shardId(), shardBytesUsed);
}
}
queue.add(new ShardAndBytesUsed(shardBytesUsed, shard));
}
}
logger.debug("now write some indexing buffers: total indexing heap bytes used [{}] vs {} [{}], currently writing bytes [{}], [{}] shards with non-zero indexing buffer",
new ByteSizeValue(totalBytesUsed), INDEX_BUFFER_SIZE_SETTING, indexingBuffer, new ByteSizeValue(totalBytesWriting), queue.size());
while (totalBytesUsed > indexingBuffer.bytes() && queue.isEmpty() == false) {
ShardAndBytesUsed largest = queue.poll();
logger.debug("write indexing buffer to disk for shard [{}] to free up its [{}] indexing buffer", largest.shard.shardId(), new ByteSizeValue(largest.bytesUsed));
writeIndexingBufferAsync(largest.shard);
totalBytesUsed -= largest.bytesUsed;
if (doThrottle && throttled.contains(largest.shard) == false) {
logger.info("now throttling indexing for shard [{}]: segment writing can't keep up", largest.shard.shardId());
throttled.add(largest.shard);
activateThrottling(largest.shard);
}
}
}
if (doThrottle == false) {
for(IndexShard shard : throttled) {
logger.info("stop throttling indexing for shard [{}]", shard.shardId());
deactivateThrottling(shard);
}
throttled.clear();
}
}
}
/**
* ask this shard to check now whether it is inactive, and reduces its indexing buffer if so.
*/
protected void checkIdle(IndexShard shard, long inactiveTimeNS) {
try {
shard.checkIdle(inactiveTimeNS);
} catch (EngineClosedException | FlushNotAllowedEngineException e) {
logger.trace("ignore exception while checking if shard {} is inactive", e, shard.shardId());
}
}
}

View File

@ -293,14 +293,13 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, analysisRegistry);
pluginsService.onIndexModule(indexModule);
indexModule.addIndexEventListener(indexingMemoryController);
for (IndexEventListener listener : builtInListeners) {
indexModule.addIndexEventListener(listener);
}
indexModule.addIndexEventListener(oldShardsStats);
final IndexEventListener listener = indexModule.freeze();
listener.beforeIndexCreated(index, idxSettings.getSettings());
final IndexService indexService = indexModule.newIndexService(nodeEnv, this, nodeServicesProvider, mapperRegistry);
final IndexService indexService = indexModule.newIndexService(nodeEnv, this, nodeServicesProvider, mapperRegistry, indexingMemoryController);
boolean success = false;
try {
assert indexService.getIndexEventListener() == listener;

View File

@ -73,15 +73,10 @@ import java.util.Locale;
*/
public enum PreBuiltAnalyzers {
STANDARD(CachingStrategy.ELASTICSEARCH) { // we don't do stopwords anymore from 1.0Beta on
STANDARD(CachingStrategy.ELASTICSEARCH) {
@Override
protected Analyzer create(Version version) {
final Analyzer a;
if (version.onOrAfter(Version.V_1_0_0_Beta1)) {
a = new StandardAnalyzer(CharArraySet.EMPTY_SET);
} else {
a = new StandardAnalyzer();
}
final Analyzer a = new StandardAnalyzer(CharArraySet.EMPTY_SET);
a.setVersion(version.luceneVersion);
return a;
}
@ -151,22 +146,14 @@ public enum PreBuiltAnalyzers {
PATTERN(CachingStrategy.ELASTICSEARCH) {
@Override
protected Analyzer create(Version version) {
if (version.onOrAfter(Version.V_1_0_0_RC1)) {
return new PatternAnalyzer(Regex.compile("\\W+" /*PatternAnalyzer.NON_WORD_PATTERN*/, null), true, CharArraySet.EMPTY_SET);
}
return new PatternAnalyzer(Regex.compile("\\W+" /*PatternAnalyzer.NON_WORD_PATTERN*/, null), true, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
return new PatternAnalyzer(Regex.compile("\\W+" /*PatternAnalyzer.NON_WORD_PATTERN*/, null), true, CharArraySet.EMPTY_SET);
}
},
STANDARD_HTML_STRIP(CachingStrategy.ELASTICSEARCH) {
@Override
protected Analyzer create(Version version) {
final Analyzer analyzer;
if (version.onOrAfter(Version.V_1_0_0_RC1)) {
analyzer = new StandardHtmlStripAnalyzer(CharArraySet.EMPTY_SET);
} else {
analyzer = new StandardHtmlStripAnalyzer();
}
final Analyzer analyzer = new StandardHtmlStripAnalyzer(CharArraySet.EMPTY_SET);
analyzer.setVersion(version.luceneVersion);
return analyzer;
}

View File

@ -28,7 +28,6 @@ import org.apache.lucene.analysis.ckb.SoraniNormalizationFilter;
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
import org.apache.lucene.analysis.core.DecimalDigitFilter;
import org.apache.lucene.analysis.core.LowerCaseFilter;
import org.apache.lucene.analysis.core.Lucene43StopFilter;
import org.apache.lucene.analysis.core.StopAnalyzer;
import org.apache.lucene.analysis.core.StopFilter;
import org.apache.lucene.analysis.core.UpperCaseFilter;
@ -45,9 +44,6 @@ import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
import org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilter;
import org.apache.lucene.analysis.miscellaneous.LengthFilter;
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
import org.apache.lucene.analysis.miscellaneous.Lucene43LengthFilter;
import org.apache.lucene.analysis.miscellaneous.Lucene43TrimFilter;
import org.apache.lucene.analysis.miscellaneous.Lucene47WordDelimiterFilter;
import org.apache.lucene.analysis.miscellaneous.ScandinavianFoldingFilter;
import org.apache.lucene.analysis.miscellaneous.ScandinavianNormalizationFilter;
import org.apache.lucene.analysis.miscellaneous.TrimFilter;
@ -55,8 +51,6 @@ import org.apache.lucene.analysis.miscellaneous.TruncateTokenFilter;
import org.apache.lucene.analysis.miscellaneous.UniqueTokenFilter;
import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenFilter;
import org.apache.lucene.analysis.ngram.Lucene43NGramTokenFilter;
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
import org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilter;
import org.apache.lucene.analysis.payloads.TypeAsPayloadTokenFilter;
@ -86,49 +80,26 @@ public enum PreBuiltTokenFilters {
WORD_DELIMITER(CachingStrategy.ONE) {
@Override
public TokenStream create(TokenStream tokenStream, Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_8)) {
return new WordDelimiterFilter(tokenStream,
WordDelimiterFilter.GENERATE_WORD_PARTS |
WordDelimiterFilter.GENERATE_NUMBER_PARTS |
WordDelimiterFilter.SPLIT_ON_CASE_CHANGE |
WordDelimiterFilter.SPLIT_ON_NUMERICS |
WordDelimiterFilter.STEM_ENGLISH_POSSESSIVE, null);
} else {
return new Lucene47WordDelimiterFilter(tokenStream,
WordDelimiterFilter.GENERATE_WORD_PARTS |
WordDelimiterFilter.GENERATE_NUMBER_PARTS |
WordDelimiterFilter.SPLIT_ON_CASE_CHANGE |
WordDelimiterFilter.SPLIT_ON_NUMERICS |
WordDelimiterFilter.STEM_ENGLISH_POSSESSIVE, null);
}
return new WordDelimiterFilter(tokenStream,
WordDelimiterFilter.GENERATE_WORD_PARTS |
WordDelimiterFilter.GENERATE_NUMBER_PARTS |
WordDelimiterFilter.SPLIT_ON_CASE_CHANGE |
WordDelimiterFilter.SPLIT_ON_NUMERICS |
WordDelimiterFilter.STEM_ENGLISH_POSSESSIVE, null);
}
},
STOP(CachingStrategy.LUCENE) {
@Override
public TokenStream create(TokenStream tokenStream, Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_4_0)) {
return new StopFilter(tokenStream, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
} else {
@SuppressWarnings("deprecation")
final TokenStream filter = new Lucene43StopFilter(true, tokenStream, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
return filter;
}
return new StopFilter(tokenStream, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
}
},
TRIM(CachingStrategy.LUCENE) {
@Override
public TokenStream create(TokenStream tokenStream, Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_4_0)) {
return new TrimFilter(tokenStream);
} else {
@SuppressWarnings("deprecation")
final TokenStream filter = new Lucene43TrimFilter(tokenStream, true);
return filter;
}
return new TrimFilter(tokenStream);
}
},
@ -149,13 +120,7 @@ public enum PreBuiltTokenFilters {
LENGTH(CachingStrategy.LUCENE) {
@Override
public TokenStream create(TokenStream tokenStream, Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_4_0)) {
return new LengthFilter(tokenStream, 0, Integer.MAX_VALUE);
} else {
@SuppressWarnings("deprecation")
final TokenStream filter = new Lucene43LengthFilter(true, tokenStream, 0, Integer.MAX_VALUE);
return filter;
}
return new LengthFilter(tokenStream, 0, Integer.MAX_VALUE);
}
},
@ -211,26 +176,14 @@ public enum PreBuiltTokenFilters {
NGRAM(CachingStrategy.LUCENE) {
@Override
public TokenStream create(TokenStream tokenStream, Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_4_0)) {
return new NGramTokenFilter(tokenStream);
} else {
@SuppressWarnings("deprecation")
final TokenStream filter = new Lucene43NGramTokenFilter(tokenStream);
return filter;
}
return new NGramTokenFilter(tokenStream);
}
},
EDGE_NGRAM(CachingStrategy.LUCENE) {
@Override
public TokenStream create(TokenStream tokenStream, Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_4_0)) {
return new EdgeNGramTokenFilter(tokenStream, EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
} else {
@SuppressWarnings("deprecation")
final TokenStream filter = new Lucene43EdgeNGramTokenFilter(tokenStream, EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
return filter;
}
return new EdgeNGramTokenFilter(tokenStream, EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
}
},

View File

@ -24,16 +24,12 @@ import org.apache.lucene.analysis.core.LetterTokenizer;
import org.apache.lucene.analysis.core.LowerCaseTokenizer;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenizer;
import org.apache.lucene.analysis.ngram.Lucene43NGramTokenizer;
import org.apache.lucene.analysis.ngram.NGramTokenizer;
import org.apache.lucene.analysis.path.PathHierarchyTokenizer;
import org.apache.lucene.analysis.pattern.PatternTokenizer;
import org.apache.lucene.analysis.standard.ClassicTokenizer;
import org.apache.lucene.analysis.standard.StandardTokenizer;
import org.apache.lucene.analysis.standard.UAX29URLEmailTokenizer;
import org.apache.lucene.analysis.standard.std40.StandardTokenizer40;
import org.apache.lucene.analysis.standard.std40.UAX29URLEmailTokenizer40;
import org.apache.lucene.analysis.th.ThaiTokenizer;
import org.elasticsearch.Version;
import org.elasticsearch.common.regex.Regex;
@ -50,11 +46,7 @@ public enum PreBuiltTokenizers {
STANDARD(CachingStrategy.LUCENE) {
@Override
protected Tokenizer create(Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_7_0)) {
return new StandardTokenizer();
} else {
return new StandardTokenizer40();
}
return new StandardTokenizer();
}
},
@ -68,11 +60,7 @@ public enum PreBuiltTokenizers {
UAX_URL_EMAIL(CachingStrategy.LUCENE) {
@Override
protected Tokenizer create(Version version) {
if (version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_7_0)) {
return new UAX29URLEmailTokenizer();
} else {
return new UAX29URLEmailTokenizer40();
}
return new UAX29URLEmailTokenizer();
}
},
@ -114,28 +102,14 @@ public enum PreBuiltTokenizers {
NGRAM(CachingStrategy.LUCENE) {
@Override
protected Tokenizer create(Version version) {
// see NGramTokenizerFactory for an explanation of this logic:
// 4.4 patch was used before 4.4 was released
if (version.onOrAfter(org.elasticsearch.Version.V_0_90_2) &&
version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_3)) {
return new NGramTokenizer();
} else {
return new Lucene43NGramTokenizer();
}
return new NGramTokenizer();
}
},
EDGE_NGRAM(CachingStrategy.LUCENE) {
@Override
protected Tokenizer create(Version version) {
// see EdgeNGramTokenizerFactory for an explanation of this logic:
// 4.4 patch was used before 4.4 was released
if (version.onOrAfter(org.elasticsearch.Version.V_0_90_2) &&
version.luceneVersion.onOrAfter(org.apache.lucene.util.Version.LUCENE_4_3)) {
return new EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE);
} else {
return new Lucene43EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE);
}
return new EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE);
}
},

View File

@ -20,11 +20,16 @@
package org.elasticsearch.monitor.os;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.SuppressForbidden;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.monitor.Probes;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.OperatingSystemMXBean;
import java.lang.reflect.Method;
import java.nio.file.Files;
import java.util.List;
public class OsProbe {
@ -103,19 +108,42 @@ public class OsProbe {
}
/**
* Returns the system load average for the last minute.
* Returns the system load averages
*/
public double getSystemLoadAverage() {
public double[] getSystemLoadAverage() {
if (Constants.LINUX) {
double[] loadAverage = readProcLoadavg("/proc/loadavg");
if (loadAverage != null) {
return loadAverage;
}
// fallback
}
if (getSystemLoadAverage == null) {
return -1;
return null;
}
try {
return (double) getSystemLoadAverage.invoke(osMxBean);
double oneMinuteLoadAverage = (double) getSystemLoadAverage.invoke(osMxBean);
return new double[] { oneMinuteLoadAverage, -1, -1 };
} catch (Throwable t) {
return -1;
return null;
}
}
@SuppressForbidden(reason = "access /proc")
private static double[] readProcLoadavg(String procLoadavg) {
try {
List<String> lines = Files.readAllLines(PathUtils.get(procLoadavg));
if (!lines.isEmpty()) {
String[] fields = lines.get(0).split("\\s+");
return new double[] { Double.parseDouble(fields[0]), Double.parseDouble(fields[1]), Double.parseDouble(fields[2]) };
}
} catch (IOException e) {
// do not fail Elasticsearch if something unexpected
// happens here
}
return null;
}
public short getSystemCpuPercent() {
return Probes.getLoadAndScaleToPercent(getSystemCpuLoad, osMxBean);
}

View File

@ -87,7 +87,13 @@ public class OsStats implements Streamable, ToXContent {
if (cpu != null) {
builder.startObject(Fields.CPU);
builder.field(Fields.PERCENT, cpu.getPercent());
builder.field(Fields.LOAD_AVERAGE, cpu.getLoadAverage());
if (cpu.getLoadAverage() != null) {
builder.startArray(Fields.LOAD_AVERAGE);
builder.value(cpu.getLoadAverage()[0]);
builder.value(cpu.getLoadAverage()[1]);
builder.value(cpu.getLoadAverage()[2]);
builder.endArray();
}
builder.endObject();
}
@ -152,8 +158,9 @@ public class OsStats implements Streamable, ToXContent {
}
public static class Cpu implements Streamable {
short percent = -1;
double loadAverage = -1;
double[] loadAverage = null;
Cpu() {}
@ -166,20 +173,29 @@ public class OsStats implements Streamable, ToXContent {
@Override
public void readFrom(StreamInput in) throws IOException {
percent = in.readShort();
loadAverage = in.readDouble();
if (in.readBoolean()) {
loadAverage = in.readDoubleArray();
} else {
loadAverage = null;
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeShort(percent);
out.writeDouble(loadAverage);
if (loadAverage == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeDoubleArray(loadAverage);
}
}
public short getPercent() {
return percent;
}
public double getLoadAverage() {
public double[] getLoadAverage() {
return loadAverage;
}
}

View File

@ -134,7 +134,9 @@ public class RestNodesAction extends AbstractCatAction {
table.addCell("file_desc.max", "default:false;alias:fdm,fileDescriptorMax;text-align:right;desc:max file descriptors");
table.addCell("cpu", "alias:cpu;text-align:right;desc:recent cpu usage");
table.addCell("load", "alias:l;text-align:right;desc:most recent load avg");
table.addCell("load_1m", "alias:l;text-align:right;desc:1m load avg");
table.addCell("load_5m", "alias:l;text-align:right;desc:5m load avg");
table.addCell("load_15m", "alias:l;text-align:right;desc:15m load avg");
table.addCell("uptime", "default:false;alias:u;text-align:right;desc:node uptime");
table.addCell("node.role", "alias:r,role,dc,nodeRole;desc:d:data node, c:client node");
table.addCell("master", "alias:m;desc:m:master-eligible, *:current master");
@ -263,7 +265,10 @@ public class RestNodesAction extends AbstractCatAction {
table.addCell(processStats == null ? null : processStats.getMaxFileDescriptors());
table.addCell(osStats == null ? null : Short.toString(osStats.getCpu().getPercent()));
table.addCell(osStats == null ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()));
boolean hasLoadAverage = osStats != null && osStats.getCpu().getLoadAverage() != null;
table.addCell(!hasLoadAverage ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[0]));
table.addCell(!hasLoadAverage ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[1]));
table.addCell(!hasLoadAverage ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[2]));
table.addCell(jvmStats == null ? null : jvmStats.getUptime());
table.addCell(node.clientNode() ? "c" : node.dataNode() ? "d" : "-");
table.addCell(masterId == null ? "x" : masterId.equals(node.id()) ? "*" : node.masterNode() ? "m" : "-");

View File

@ -115,4 +115,7 @@ grant {
// needed by JDKESLoggerTests
permission java.util.logging.LoggingPermission "control";
// load averages on Linux
permission java.io.FilePermission "/proc/loadavg", "read";
};

View File

@ -51,6 +51,7 @@ import org.elasticsearch.index.similarity.SimilarityProvider;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.breaker.CircuitBreakerService;

View File

@ -23,11 +23,7 @@ import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenFilter;
import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenizer;
import org.apache.lucene.analysis.ngram.Lucene43NGramTokenizer;
import org.apache.lucene.analysis.ngram.NGramTokenizer;
import org.apache.lucene.analysis.reverse.ReverseStringFilter;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -118,79 +114,6 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
new String[] {" a", " a!"});
}
public void testBackwardsCompatibilityEdgeNgramTokenizer() throws Exception {
int iters = scaledRandomIntBetween(20, 100);
final Index index = new Index("test");
final String name = "ngr";
for (int i = 0; i < iters; i++) {
Version v = randomVersion(random());
if (v.onOrAfter(Version.V_0_90_2)) {
Builder builder = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit");
boolean compatVersion = false;
if ((compatVersion = random().nextBoolean())) {
builder.put("version", "4." + random().nextInt(3));
builder.put("side", "back");
}
Settings settings = builder.build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer edgeNGramTokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
edgeNGramTokenizer.setReader(new StringReader("foo bar"));
if (compatVersion) {
assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class));
} else {
assertThat(edgeNGramTokenizer, instanceOf(EdgeNGramTokenizer.class));
}
} else {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("side", "back").build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer edgeNGramTokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
edgeNGramTokenizer.setReader(new StringReader("foo bar"));
assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class));
}
}
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("side", "back").build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
try {
new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
fail("should fail side:back is not supported anymore");
} catch (IllegalArgumentException ex) {
}
}
public void testBackwardsCompatibilityNgramTokenizer() throws Exception {
int iters = scaledRandomIntBetween(20, 100);
for (int i = 0; i < iters; i++) {
final Index index = new Index("test");
final String name = "ngr";
Version v = randomVersion(random());
if (v.onOrAfter(Version.V_0_90_2)) {
Builder builder = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit");
boolean compatVersion = false;
if ((compatVersion = random().nextBoolean())) {
builder.put("version", "4." + random().nextInt(3));
}
Settings settings = builder.build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer nGramTokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
nGramTokenizer.setReader(new StringReader("foo bar"));
if (compatVersion) {
assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class));
} else {
assertThat(nGramTokenizer, instanceOf(NGramTokenizer.class));
}
} else {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer nGramTokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
nGramTokenizer.setReader(new StringReader("foo bar"));
assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class));
}
}
}
public void testBackwardsCompatibilityEdgeNgramTokenFilter() throws Exception {
int iters = scaledRandomIntBetween(20, 100);
for (int i = 0; i < iters; i++) {

View File

@ -19,8 +19,6 @@
package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.compress.CompressedXContent;
@ -32,15 +30,11 @@ import org.elasticsearch.indices.analysis.PreBuiltAnalyzers;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import static org.elasticsearch.test.VersionUtils.randomVersion;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
/**
*
@ -54,76 +48,6 @@ public class PreBuiltAnalyzerTests extends ESSingleNodeTestCase {
assertThat(currentDefaultAnalyzer, is(currentStandardAnalyzer));
}
public void testThatDefaultAndStandardAnalyzerChangedIn10Beta1() throws IOException {
Analyzer currentStandardAnalyzer = PreBuiltAnalyzers.STANDARD.getAnalyzer(Version.V_1_0_0_Beta1);
Analyzer currentDefaultAnalyzer = PreBuiltAnalyzers.DEFAULT.getAnalyzer(Version.V_1_0_0_Beta1);
// special case, these two are the same instance
assertThat(currentDefaultAnalyzer, is(currentStandardAnalyzer));
PreBuiltAnalyzers.DEFAULT.getAnalyzer(Version.V_1_0_0_Beta1);
final int n = scaledRandomIntBetween(10, 100);
Version version = Version.CURRENT;
for(int i = 0; i < n; i++) {
if (version.equals(Version.V_1_0_0_Beta1)) {
assertThat(currentDefaultAnalyzer, is(PreBuiltAnalyzers.DEFAULT.getAnalyzer(version)));
} else {
assertThat(currentDefaultAnalyzer, not(is(PreBuiltAnalyzers.DEFAULT.getAnalyzer(version))));
}
Analyzer analyzer = PreBuiltAnalyzers.DEFAULT.getAnalyzer(version);
TokenStream ts = analyzer.tokenStream("foo", "This is it Dude");
ts.reset();
CharTermAttribute charTermAttribute = ts.addAttribute(CharTermAttribute.class);
List<String> list = new ArrayList<>();
while(ts.incrementToken()) {
list.add(charTermAttribute.toString());
}
if (version.onOrAfter(Version.V_1_0_0_Beta1)) {
assertThat(list.size(), is(4));
assertThat(list, contains("this", "is", "it", "dude"));
} else {
assertThat(list.size(), is(1));
assertThat(list, contains("dude"));
}
ts.close();
version = randomVersion(random());
}
}
public void testAnalyzerChangedIn10RC1() throws IOException {
Analyzer pattern = PreBuiltAnalyzers.PATTERN.getAnalyzer(Version.V_1_0_0_RC1);
Analyzer standardHtml = PreBuiltAnalyzers.STANDARD_HTML_STRIP.getAnalyzer(Version.V_1_0_0_RC1);
final int n = scaledRandomIntBetween(10, 100);
Version version = Version.CURRENT;
for(int i = 0; i < n; i++) {
if (version.equals(Version.V_1_0_0_RC1)) {
assertThat(pattern, is(PreBuiltAnalyzers.PATTERN.getAnalyzer(version)));
assertThat(standardHtml, is(PreBuiltAnalyzers.STANDARD_HTML_STRIP.getAnalyzer(version)));
} else {
assertThat(pattern, not(is(PreBuiltAnalyzers.DEFAULT.getAnalyzer(version))));
assertThat(standardHtml, not(is(PreBuiltAnalyzers.DEFAULT.getAnalyzer(version))));
}
Analyzer analyzer = randomBoolean() ? PreBuiltAnalyzers.PATTERN.getAnalyzer(version) : PreBuiltAnalyzers.STANDARD_HTML_STRIP.getAnalyzer(version);
TokenStream ts = analyzer.tokenStream("foo", "This is it Dude");
ts.reset();
CharTermAttribute charTermAttribute = ts.addAttribute(CharTermAttribute.class);
List<String> list = new ArrayList<>();
while(ts.incrementToken()) {
list.add(charTermAttribute.toString());
}
if (version.onOrAfter(Version.V_1_0_0_RC1)) {
assertThat(list.toString(), list.size(), is(4));
assertThat(list, contains("this", "is", "it", "dude"));
} else {
assertThat(list.size(), is(1));
assertThat(list, contains("dude"));
}
ts.close();
version = randomVersion(random());
}
}
public void testThatInstancesAreTheSameAlwaysForKeywordAnalyzer() {
assertThat(PreBuiltAnalyzers.KEYWORD.getAnalyzer(Version.CURRENT),
is(PreBuiltAnalyzers.KEYWORD.getAnalyzer(Version.V_0_18_0)));

View File

@ -40,8 +40,6 @@ public class InternalEngineSettingsTests extends ESSingleNodeTestCase {
// VERSION MAP SIZE
long indexBufferSize = engine.config().getIndexingBufferSize().bytes();
long versionMapSize = engine.config().getVersionMapSize().bytes();
assertThat(versionMapSize, equalTo((long) (indexBufferSize * 0.25)));
final int iters = between(1, 20);
for (int i = 0; i < iters; i++) {
@ -51,14 +49,8 @@ public class InternalEngineSettingsTests extends ESSingleNodeTestCase {
// the full long range here else the assert below fails:
long gcDeletes = random().nextLong() & (Long.MAX_VALUE >> 11);
boolean versionMapAsPercent = randomBoolean();
double versionMapPercent = randomIntBetween(0, 100);
long versionMapSizeInMB = randomIntBetween(10, 20);
String versionMapString = versionMapAsPercent ? versionMapPercent + "%" : versionMapSizeInMB + "mb";
Settings build = Settings.builder()
.put(EngineConfig.INDEX_GC_DELETES_SETTING, gcDeletes, TimeUnit.MILLISECONDS)
.put(EngineConfig.INDEX_VERSION_MAP_SIZE, versionMapString)
.build();
assertEquals(gcDeletes, build.getAsTime(EngineConfig.INDEX_GC_DELETES_SETTING, null).millis());
@ -71,12 +63,6 @@ public class InternalEngineSettingsTests extends ESSingleNodeTestCase {
assertEquals(engine.getGcDeletesInMillis(), gcDeletes);
indexBufferSize = engine.config().getIndexingBufferSize().bytes();
versionMapSize = engine.config().getVersionMapSize().bytes();
if (versionMapAsPercent) {
assertThat(versionMapSize, equalTo((long) (indexBufferSize * (versionMapPercent / 100))));
} else {
assertThat(versionMapSize, equalTo(1024 * 1024 * versionMapSizeInMB));
}
}
Settings settings = Settings.builder()
@ -101,37 +87,5 @@ public class InternalEngineSettingsTests extends ESSingleNodeTestCase {
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
assertEquals(engine.getGcDeletesInMillis(), 1000);
assertTrue(engine.config().isEnableGcDeletes());
settings = Settings.builder()
.put(EngineConfig.INDEX_VERSION_MAP_SIZE, "sdfasfd")
.build();
try {
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
fail("settings update didn't fail, but should have");
} catch (IllegalArgumentException e) {
// good
}
settings = Settings.builder()
.put(EngineConfig.INDEX_VERSION_MAP_SIZE, "-12%")
.build();
try {
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
fail("settings update didn't fail, but should have");
} catch (IllegalArgumentException e) {
// good
}
settings = Settings.builder()
.put(EngineConfig.INDEX_VERSION_MAP_SIZE, "130%")
.build();
try {
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
fail("settings update didn't fail, but should have");
} catch (IllegalArgumentException e) {
// good
}
}
}

View File

@ -95,6 +95,7 @@ import org.elasticsearch.index.store.DirectoryUtils;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.test.DummyShardLock;
@ -439,7 +440,7 @@ public class InternalEngineTests extends ESTestCase {
public void testSegmentsWithMergeFlag() throws Exception {
try (Store store = createStore();
Engine engine = createEngine(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), new TieredMergePolicy())) {
Engine engine = createEngine(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), new TieredMergePolicy())) {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(newUid("1"), doc);
engine.index(index);
@ -769,7 +770,7 @@ public class InternalEngineTests extends ESTestCase {
public void testSyncedFlush() throws IOException {
try (Store store = createStore();
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings),
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings),
new LogByteSizeMergePolicy()), false)) {
final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20);
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
@ -1026,7 +1027,7 @@ public class InternalEngineTests extends ESTestCase {
public void testForceMerge() throws IOException {
try (Store store = createStore();
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings),
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings),
new LogByteSizeMergePolicy()), false)) { // use log MP here we test some behavior in ESMP
int numDocs = randomIntBetween(10, 100);
for (int i = 0; i < numDocs; i++) {
@ -1465,7 +1466,7 @@ public class InternalEngineTests extends ESTestCase {
public void testEnableGcDeletes() throws Exception {
try (Store store = createStore();
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()), false)) {
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()), false)) {
engine.config().setEnableGcDeletes(false);
// Add document
@ -1586,44 +1587,6 @@ public class InternalEngineTests extends ESTestCase {
assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName());
}
// #10312
public void testDeletesAloneCanTriggerRefresh() throws Exception {
try (Store store = createStore();
Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()),
false)) {
engine.config().setIndexingBufferSize(new ByteSizeValue(1, ByteSizeUnit.KB));
for (int i = 0; i < 100; i++) {
String id = Integer.toString(i);
ParsedDocument doc = testParsedDocument(id, id, "test", null, -1, -1, testDocument(), B_1, null);
engine.index(new Engine.Index(newUid(id), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()));
}
// Force merge so we know all merges are done before we start deleting:
engine.forceMerge(true, 1, false, false, false);
Searcher s = engine.acquireSearcher("test");
final long version1 = ((DirectoryReader) s.reader()).getVersion();
s.close();
for (int i = 0; i < 100; i++) {
String id = Integer.toString(i);
engine.delete(new Engine.Delete("test", id, newUid(id), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false));
}
// We must assertBusy because refresh due to version map being full is done in background (REFRESH) thread pool:
assertBusy(new Runnable() {
@Override
public void run() {
Searcher s2 = engine.acquireSearcher("test");
long version2 = ((DirectoryReader) s2.reader()).getVersion();
s2.close();
// 100 buffered deletes will easily exceed 25% of our 1 KB indexing buffer so it should have forced a refresh:
assertThat(version2, greaterThan(version1));
}
});
}
}
public void testMissingTranslog() throws IOException {
// test that we can force start the engine , even if the translog is missing.
engine.close();

View File

@ -367,12 +367,13 @@ public class IndexShardTests extends ESSingleNodeTestCase {
client().prepareIndex("test", "test").setSource("{}").get();
ensureGreen("test");
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
Boolean result = indicesService.indexService("test").getShardOrNull(0).checkIdle(0);
assertEquals(Boolean.TRUE, result);
indicesService.indexService("test").getShardOrNull(0).checkIdle(0);
assertBusy(() -> {
IndexStats indexStats = client().admin().indices().prepareStats("test").clear().get().getIndex("test");
assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID));
});
indicesService.indexService("test").getShardOrNull(0).checkIdle(0);
}
);
IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test");
assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID));
}

View File

@ -1,99 +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.indices;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.test.ESIntegTestCase;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class IndexingMemoryControllerIT extends ESIntegTestCase {
private long getIWBufferSize(String indexName) {
return client().admin().indices().prepareStats(indexName).get().getTotal().getSegments().getIndexWriterMaxMemoryInBytes();
}
public void testIndexBufferPushedToEngine() throws InterruptedException {
createNode(Settings.builder().put(IndexShard.INDEX_SHARD_INACTIVE_TIME_SETTING, "100000h",
IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "32mb",
IndexShard.INDEX_REFRESH_INTERVAL, "-1").build());
// Create two active indices, sharing 32 MB indexing buffer:
prepareCreate("test3").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).get();
prepareCreate("test4").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).get();
ensureGreen();
index("test3", "type", "1", "f", 1);
index("test4", "type", "1", "f", 1);
// .. then make sure we really pushed the update (16 MB for each) down to the IndexWriter, even if refresh nor flush occurs:
if (awaitBusy(() -> getIWBufferSize("test3") == 16*1024*1024) == false) {
fail("failed to update shard indexing buffer size for test3 index to 16 MB; got: " + getIWBufferSize("test3"));
}
if (awaitBusy(() -> getIWBufferSize("test4") == 16*1024*1024) == false) {
fail("failed to update shard indexing buffer size for test4 index to 16 MB; got: " + getIWBufferSize("test4"));
}
client().admin().indices().prepareDelete("test4").get();
if (awaitBusy(() -> getIWBufferSize("test3") == 32 * 1024 * 1024) == false) {
fail("failed to update shard indexing buffer size for test3 index to 32 MB; got: " + getIWBufferSize("test4"));
}
}
public void testInactivePushedToShard() throws InterruptedException {
createNode(Settings.builder().put(IndexShard.INDEX_SHARD_INACTIVE_TIME_SETTING, "100ms",
IndexingMemoryController.SHARD_INACTIVE_INTERVAL_TIME_SETTING, "100ms",
IndexShard.INDEX_REFRESH_INTERVAL, "-1").build());
// Create two active indices, sharing 32 MB indexing buffer:
prepareCreate("test1").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).get();
ensureGreen();
index("test1", "type", "1", "f", 1);
// make shard the shard buffer was set to inactive size
final ByteSizeValue inactiveBuffer = IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER;
if (awaitBusy(() -> getIWBufferSize("test1") == inactiveBuffer.bytes()) == false) {
fail("failed to update shard indexing buffer size for test1 index to [" + inactiveBuffer + "]; got: " + getIWBufferSize("test1"));
}
}
private void createNode(Settings settings) {
internalCluster().startNode(Settings.builder()
.put(ClusterName.SETTING, "IndexingMemoryControllerIT")
.put("node.name", "IndexingMemoryControllerIT")
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put(EsExecutors.PROCESSORS, 1) // limit the number of threads created
.put("http.enabled", false)
.put(InternalSettingsPreparer.IGNORE_SYSTEM_PROPERTIES_SETTING, true) // make sure we get what we set :)
.put(IndexingMemoryController.SHARD_INACTIVE_INTERVAL_TIME_SETTING, "100ms")
.put(settings)
);
}
}

View File

@ -18,16 +18,20 @@
*/
package org.elasticsearch.indices;
import org.apache.lucene.index.DirectoryReader;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -37,88 +41,114 @@ import java.util.concurrent.ScheduledFuture;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.equalTo;
public class IndexingMemoryControllerTests extends ESSingleNodeTestCase {
static class MockController extends IndexingMemoryController {
final static ByteSizeValue INACTIVE = new ByteSizeValue(-1);
// Size of each shard's indexing buffer
final Map<IndexShard, Long> indexBufferRAMBytesUsed = new HashMap<>();
final Map<IndexShard, ByteSizeValue> indexingBuffers = new HashMap<>();
// How many bytes this shard is currently moving to disk
final Map<IndexShard, Long> writingBytes = new HashMap<>();
final Map<IndexShard, Long> lastIndexTimeNanos = new HashMap<>();
final Set<IndexShard> activeShards = new HashSet<>();
long currentTimeSec = TimeValue.timeValueNanos(System.nanoTime()).seconds();
// Shards that are currently throttled
final Set<IndexShard> throttled = new HashSet<>();
public MockController(Settings settings) {
super(Settings.builder()
.put(SHARD_INACTIVE_INTERVAL_TIME_SETTING, "200h") // disable it
.put(IndexShard.INDEX_SHARD_INACTIVE_TIME_SETTING, "1ms") // nearly immediate
.put(settings)
.build(),
null, null, 100 * 1024 * 1024); // fix jvm mem size to 100mb
.put(SHARD_MEMORY_INTERVAL_TIME_SETTING, "200h") // disable it
.put(settings)
.build(),
null, null, 100 * 1024 * 1024); // fix jvm mem size to 100mb
}
public void deleteShard(IndexShard id) {
indexingBuffers.remove(id);
}
public void assertBuffers(IndexShard id, ByteSizeValue indexing) {
assertThat(indexingBuffers.get(id), equalTo(indexing));
}
public void assertInactive(IndexShard id) {
assertThat(indexingBuffers.get(id), equalTo(INACTIVE));
}
@Override
protected long currentTimeInNanos() {
return TimeValue.timeValueSeconds(currentTimeSec).nanos();
public void deleteShard(IndexShard shard) {
indexBufferRAMBytesUsed.remove(shard);
writingBytes.remove(shard);
}
@Override
protected List<IndexShard> availableShards() {
return new ArrayList<>(indexingBuffers.keySet());
return new ArrayList<>(indexBufferRAMBytesUsed.keySet());
}
@Override
protected boolean shardAvailable(IndexShard shard) {
return indexingBuffers.containsKey(shard);
protected long getIndexBufferRAMBytesUsed(IndexShard shard) {
return indexBufferRAMBytesUsed.get(shard) + writingBytes.get(shard);
}
@Override
protected void updateShardBuffers(IndexShard shard, ByteSizeValue shardIndexingBufferSize) {
indexingBuffers.put(shard, shardIndexingBufferSize);
}
@Override
protected boolean checkIdle(IndexShard shard) {
final TimeValue inactiveTime = settings.getAsTime(IndexShard.INDEX_SHARD_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5));
Long ns = lastIndexTimeNanos.get(shard);
if (ns == null) {
return true;
} else if (currentTimeInNanos() - ns >= inactiveTime.nanos()) {
indexingBuffers.put(shard, INACTIVE);
activeShards.remove(shard);
return true;
protected long getShardWritingBytes(IndexShard shard) {
Long bytes = writingBytes.get(shard);
if (bytes == null) {
return 0;
} else {
return false;
return bytes;
}
}
public void incrementTimeSec(int sec) {
currentTimeSec += sec;
@Override
protected void checkIdle(IndexShard shard, long inactiveTimeNS) {
}
@Override
public void writeIndexingBufferAsync(IndexShard shard) {
long bytes = indexBufferRAMBytesUsed.put(shard, 0L);
writingBytes.put(shard, writingBytes.get(shard) + bytes);
indexBufferRAMBytesUsed.put(shard, 0L);
}
@Override
public void activateThrottling(IndexShard shard) {
assertTrue(throttled.add(shard));
}
@Override
public void deactivateThrottling(IndexShard shard) {
assertTrue(throttled.remove(shard));
}
public void doneWriting(IndexShard shard) {
writingBytes.put(shard, 0L);
}
public void assertBuffer(IndexShard shard, int expectedMB) {
Long actual = indexBufferRAMBytesUsed.get(shard);
if (actual == null) {
actual = 0L;
}
assertEquals(expectedMB * 1024 * 1024, actual.longValue());
}
public void assertThrottled(IndexShard shard) {
assertTrue(throttled.contains(shard));
}
public void assertNotThrottled(IndexShard shard) {
assertFalse(throttled.contains(shard));
}
public void assertWriting(IndexShard shard, int expectedMB) {
Long actual = writingBytes.get(shard);
if (actual == null) {
actual = 0L;
}
assertEquals(expectedMB * 1024 * 1024, actual.longValue());
}
public void simulateIndexing(IndexShard shard) {
lastIndexTimeNanos.put(shard, currentTimeInNanos());
if (indexingBuffers.containsKey(shard) == false) {
// First time we are seeing this shard; start it off with inactive buffers as IndexShard does:
indexingBuffers.put(shard, IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER);
Long bytes = indexBufferRAMBytesUsed.get(shard);
if (bytes == null) {
bytes = 0L;
// First time we are seeing this shard:
writingBytes.put(shard, 0L);
}
activeShards.add(shard);
// Each doc we index takes up a megabyte!
bytes += 1024*1024;
indexBufferRAMBytesUsed.put(shard, bytes);
forceCheck();
}
@ -134,21 +164,21 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase {
IndexService test = indicesService.indexService("test");
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb").build());
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "4mb").build());
IndexShard shard0 = test.getShard(0);
controller.simulateIndexing(shard0);
controller.assertBuffers(shard0, new ByteSizeValue(10, ByteSizeUnit.MB)); // translog is maxed at 64K
controller.assertBuffer(shard0, 1);
// add another shard
IndexShard shard1 = test.getShard(1);
controller.simulateIndexing(shard1);
controller.assertBuffers(shard0, new ByteSizeValue(5, ByteSizeUnit.MB));
controller.assertBuffers(shard1, new ByteSizeValue(5, ByteSizeUnit.MB));
controller.assertBuffer(shard0, 1);
controller.assertBuffer(shard1, 1);
// remove first shard
controller.deleteShard(shard0);
controller.forceCheck();
controller.assertBuffers(shard1, new ByteSizeValue(10, ByteSizeUnit.MB)); // translog is maxed at 64K
controller.assertBuffer(shard1, 1);
// remove second shard
controller.deleteShard(shard1);
@ -157,85 +187,48 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase {
// add a new one
IndexShard shard2 = test.getShard(2);
controller.simulateIndexing(shard2);
controller.assertBuffers(shard2, new ByteSizeValue(10, ByteSizeUnit.MB)); // translog is maxed at 64K
controller.assertBuffer(shard2, 1);
}
public void testActiveInactive() {
createIndex("test", Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 2).put(SETTING_NUMBER_OF_REPLICAS, 0).build());
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService test = indicesService.indexService("test");
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb")
.put(IndexShard.INDEX_SHARD_INACTIVE_TIME_SETTING, "5s")
.build());
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "5mb")
.build());
IndexShard shard0 = test.getShard(0);
controller.simulateIndexing(shard0);
IndexShard shard1 = test.getShard(1);
controller.simulateIndexing(shard1);
controller.assertBuffers(shard0, new ByteSizeValue(5, ByteSizeUnit.MB));
controller.assertBuffers(shard1, new ByteSizeValue(5, ByteSizeUnit.MB));
// index into both shards, move the clock and see that they are still active
controller.assertBuffer(shard0, 1);
controller.assertBuffer(shard1, 1);
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard1);
controller.incrementTimeSec(10);
controller.forceCheck();
controller.assertBuffer(shard0, 2);
controller.assertBuffer(shard1, 2);
// both shards now inactive
controller.assertInactive(shard0);
controller.assertInactive(shard1);
// index into one shard only, see it becomes active
// index into one shard only, crosses the 5mb limit, so shard1 is refreshed
controller.simulateIndexing(shard0);
controller.assertBuffers(shard0, new ByteSizeValue(10, ByteSizeUnit.MB));
controller.assertInactive(shard1);
controller.simulateIndexing(shard0);
controller.assertBuffer(shard0, 0);
controller.assertBuffer(shard1, 2);
controller.incrementTimeSec(3); // increment but not enough to become inactive
controller.forceCheck();
controller.assertBuffers(shard0, new ByteSizeValue(10, ByteSizeUnit.MB));
controller.assertInactive(shard1);
controller.incrementTimeSec(3); // increment some more
controller.forceCheck();
controller.assertInactive(shard0);
controller.assertInactive(shard1);
// index some and shard becomes immediately active
controller.simulateIndexing(shard1);
controller.assertInactive(shard0);
controller.assertBuffers(shard1, new ByteSizeValue(10, ByteSizeUnit.MB));
controller.simulateIndexing(shard1);
controller.assertBuffer(shard1, 4);
controller.simulateIndexing(shard1);
controller.simulateIndexing(shard1);
// shard1 crossed 5 mb and is now cleared:
controller.assertBuffer(shard1, 0);
}
public void testMinShardBufferSizes() {
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb")
.put(IndexingMemoryController.MIN_SHARD_INDEX_BUFFER_SIZE_SETTING, "6mb")
.put(IndexingMemoryController.MIN_SHARD_TRANSLOG_BUFFER_SIZE_SETTING, "40kb").build());
assertTwoActiveShards(controller, new ByteSizeValue(6, ByteSizeUnit.MB), new ByteSizeValue(40, ByteSizeUnit.KB));
}
public void testMaxShardBufferSizes() {
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb")
.put(IndexingMemoryController.MAX_SHARD_INDEX_BUFFER_SIZE_SETTING, "3mb")
.put(IndexingMemoryController.MAX_SHARD_TRANSLOG_BUFFER_SIZE_SETTING, "10kb").build());
assertTwoActiveShards(controller, new ByteSizeValue(3, ByteSizeUnit.MB), new ByteSizeValue(10, ByteSizeUnit.KB));
}
public void testRelativeBufferSizes() {
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "50%")
.build());
assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(50, ByteSizeUnit.MB)));
}
public void testMinBufferSizes() {
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "0.001%")
@ -246,21 +239,142 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase {
public void testMaxBufferSizes() {
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "90%")
.put(IndexingMemoryController.MAX_INDEX_BUFFER_SIZE_SETTING, "6mb").build());
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "90%")
.put(IndexingMemoryController.MAX_INDEX_BUFFER_SIZE_SETTING, "6mb").build());
assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(6, ByteSizeUnit.MB)));
}
protected void assertTwoActiveShards(MockController controller, ByteSizeValue indexBufferSize, ByteSizeValue translogBufferSize) {
createIndex("test", Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 2).put(SETTING_NUMBER_OF_REPLICAS, 0).build());
public void testThrottling() throws Exception {
createIndex("test", Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 3).put(SETTING_NUMBER_OF_REPLICAS, 0).build());
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService test = indicesService.indexService("test");
MockController controller = new MockController(Settings.builder()
.put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "4mb").build());
IndexShard shard0 = test.getShard(0);
controller.simulateIndexing(shard0);
IndexShard shard1 = test.getShard(1);
IndexShard shard2 = test.getShard(2);
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard0);
controller.assertBuffer(shard0, 3);
controller.simulateIndexing(shard1);
controller.assertBuffers(shard0, indexBufferSize);
controller.assertBuffers(shard1, indexBufferSize);
controller.simulateIndexing(shard1);
// We are now using 5 MB, so we should be writing shard0 since it's using the most heap:
controller.assertWriting(shard0, 3);
controller.assertWriting(shard1, 0);
controller.assertBuffer(shard0, 0);
controller.assertBuffer(shard1, 2);
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard1);
controller.simulateIndexing(shard1);
// Now we are still writing 3 MB (shard0), and using 5 MB index buffers, so we should now 1) be writing shard1, and 2) be throttling shard1:
controller.assertWriting(shard0, 3);
controller.assertWriting(shard1, 4);
controller.assertBuffer(shard0, 1);
controller.assertBuffer(shard1, 0);
controller.assertNotThrottled(shard0);
controller.assertThrottled(shard1);
System.out.println("TEST: now index more");
// More indexing to shard0
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard0);
controller.simulateIndexing(shard0);
// Now we are using 5 MB again, so shard0 should also be writing and now also be throttled:
controller.assertWriting(shard0, 8);
controller.assertWriting(shard1, 4);
controller.assertBuffer(shard0, 0);
controller.assertBuffer(shard1, 0);
controller.assertThrottled(shard0);
controller.assertThrottled(shard1);
// Both shards finally finish writing, and throttling should stop:
controller.doneWriting(shard0);
controller.doneWriting(shard1);
controller.forceCheck();
controller.assertNotThrottled(shard0);
controller.assertNotThrottled(shard1);
}
// #10312
public void testDeletesAloneCanTriggerRefresh() throws Exception {
createIndex("index",
Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1)
.put(SETTING_NUMBER_OF_REPLICAS, 0)
.put("index.refresh_interval", -1)
.build());
ensureGreen();
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService indexService = indicesService.indexService("index");
IndexShard shard = indexService.getShardOrNull(0);
assertNotNull(shard);
for (int i = 0; i < 100; i++) {
String id = Integer.toString(i);
client().prepareIndex("index", "type", id).setSource("field", "value").get();
}
// Force merge so we know all merges are done before we start deleting:
ForceMergeResponse r = client().admin().indices().prepareForceMerge().setMaxNumSegments(1).execute().actionGet();
assertNoFailures(r);
// Make a shell of an IMC to check up on indexing buffer usage:
Settings settings = Settings.builder().put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "1kb").build();
// TODO: would be cleaner if I could pass this 1kb setting to the single node this test created....
IndexingMemoryController imc = new IndexingMemoryController(settings, null, null) {
@Override
protected List<IndexShard> availableShards() {
return Collections.singletonList(shard);
}
@Override
protected long getIndexBufferRAMBytesUsed(IndexShard shard) {
return shard.getIndexBufferRAMBytesUsed();
}
@Override
protected void writeIndexingBufferAsync(IndexShard shard) {
// just do it sync'd for this test
shard.writeIndexingBuffer();
}
@Override
protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
return null;
}
};
for (int i = 0; i < 100; i++) {
String id = Integer.toString(i);
client().prepareDelete("index", "type", id).get();
}
final long indexingBufferBytes1 = shard.getIndexBufferRAMBytesUsed();
imc.forceCheck();
// We must assertBusy because the writeIndexingBufferAsync is done in background (REFRESH) thread pool:
assertBusy(new Runnable() {
@Override
public void run() {
try (Engine.Searcher s2 = shard.acquireSearcher("index")) {
// 100 buffered deletes will easily exceed our 1 KB indexing buffer so it should trigger a write:
final long indexingBufferBytes2 = shard.getIndexBufferRAMBytesUsed();
assertTrue(indexingBufferBytes2 < indexingBufferBytes1);
}
}
});
}
}

View File

@ -50,12 +50,31 @@ public class OsProbeTests extends ESTestCase {
assertNotNull(stats);
assertThat(stats.getTimestamp(), greaterThan(0L));
assertThat(stats.getCpu().getPercent(), anyOf(equalTo((short) -1), is(both(greaterThanOrEqualTo((short) 0)).and(lessThanOrEqualTo((short) 100)))));
double[] loadAverage = stats.getCpu().loadAverage;
if (loadAverage != null) {
assertThat(loadAverage.length, equalTo(3));
}
if (Constants.WINDOWS) {
// Load average is always -1 on Windows platforms
assertThat(stats.getCpu().getLoadAverage(), equalTo((double) -1));
// load average is unavailable on Windows
if (loadAverage != null) {
assertThat(loadAverage[0], equalTo((double) -1));
assertThat(loadAverage[1], equalTo((double) -1));
assertThat(loadAverage[2], equalTo((double) -1));
}
} else if (Constants.LINUX) {
// we should be able to get the load average
assertNotNull(loadAverage);
assertThat(loadAverage[0], greaterThanOrEqualTo((double) 0));
assertThat(loadAverage[1], greaterThanOrEqualTo((double) 0));
assertThat(loadAverage[2], greaterThanOrEqualTo((double) 0));
} else {
// Load average can be negative if not available or not computed yet, otherwise it should be >= 0
assertThat(stats.getCpu().getLoadAverage(), anyOf(lessThan((double) 0), greaterThanOrEqualTo((double) 0)));
// one minute load average is available, but 10-minute and 15-minute load averages are not
// load average can be negative if not available or not computed yet, otherwise it should be >= 0
if (loadAverage != null) {
assertThat(loadAverage[0], anyOf(lessThan((double) 0), greaterThanOrEqualTo((double) 0)));
assertThat(loadAverage[1], equalTo((double) -1));
assertThat(loadAverage[2], equalTo((double) -1));
}
}
assertNotNull(stats.getMem());

View File

@ -356,7 +356,8 @@ if __name__ == "__main__":
debs3_list_cmd = 'deb-s3 list -b %s --prefix %s' % (bucket, debs3_prefix)
debs3_verify_cmd = 'deb-s3 verify -b %s --prefix %s' % (bucket, debs3_prefix)
rpms3_prefix = 'elasticsearch/staging/%s-%s/repos/%s/centos' % (release_version, shortHash, package_repo_version)
rpms3_upload_cmd = 'rpm-s3 -v -b %s -p %s --sign --visibility public-read -k 100 %s' % (bucket, rpms3_prefix, rpm)
# external-1 is the alias name for the us-east-1 region. This is used by rpm-s3 to construct the hostname
rpms3_upload_cmd = 'rpm-s3 -v -b %s -p %s --sign --visibility public-read -k 100 %s -r external-1' % (bucket, rpms3_prefix, rpm)
if deploy_s3:
run(s3cmd_sync_to_staging_bucket_cmd)

View File

@ -15,6 +15,7 @@ wiki1 1 p STARTED 3013 29.6mb 192.168.56.30 Frankie Raye
wiki1 2 p STARTED 3973 38.1mb 192.168.56.20 Commander Kraken
--------------------------------------------------
[float]
[[index-pattern]]
=== Index pattern
@ -31,6 +32,7 @@ wiki2 2 p STARTED 275 7.8mb 192.168.56.20 Commander Kraken
--------------------------------------------------
[float]
[[relocation]]
=== Relocation
@ -46,6 +48,7 @@ wiki1 0 r RELOCATING 3014 31.1mb 192.168.56.20 Commander Kraken -> 192.168.56.30
wiki1 1 r RELOCATING 3013 29.6mb 192.168.56.10 Stiletto -> 192.168.56.30 Frankie Raye
--------------------------------------------------
[float]
[[states]]
=== Shard states
@ -66,8 +69,8 @@ wiki1 2 p STARTED 3973 38.1mb 192.168.56.20 Commander Kraken
--------------------------------------------------
If a shard cannot be assigned, for example you've overallocated the
number of replicas for the number of nodes in the cluster, they will
remain `UNASSIGNED`.
number of replicas for the number of nodes in the cluster, the shard
will remain `UNASSIGNED` with the <<reason-unassigned,reason code>> `ALLOCATION_FAILED`.
[source,sh]
--------------------------------------------------
@ -78,13 +81,33 @@ remain `UNASSIGNED`.
wiki1 0 p STARTED 3014 31.1mb 192.168.56.10 Stiletto
wiki1 0 r STARTED 3014 31.1mb 192.168.56.30 Frankie Raye
wiki1 0 r STARTED 3014 31.1mb 192.168.56.20 Commander Kraken
wiki1 0 r UNASSIGNED
wiki1 0 r UNASSIGNED ALLOCATION_FAILED
wiki1 1 r STARTED 3013 29.6mb 192.168.56.10 Stiletto
wiki1 1 p STARTED 3013 29.6mb 192.168.56.30 Frankie Raye
wiki1 1 r STARTED 3013 29.6mb 192.168.56.20 Commander Kraken
wiki1 1 r UNASSIGNED
wiki1 1 r UNASSIGNED ALLOCATION_FAILED
wiki1 2 r STARTED 3973 38.1mb 192.168.56.10 Stiletto
wiki1 2 r STARTED 3973 38.1mb 192.168.56.30 Frankie Raye
wiki1 2 p STARTED 3973 38.1mb 192.168.56.20 Commander Kraken
wiki1 2 r UNASSIGNED
wiki1 2 r UNASSIGNED ALLOCATION_FAILED
--------------------------------------------------
[float]
[[reason-unassigned]]
=== Reasons for unassigned shard
These are the possible reasons for a shard be in a unassigned state:
[horizontal]
`INDEX_CREATED`:: Unassigned as a result of an API creation of an index.
`CLUSTER_RECOVERED`:: Unassigned as a result of a full cluster recovery.
`INDEX_REOPENED`:: Unassigned as a result of opening a closed index.
`DANGLING_INDEX_IMPORTED`:: Unassigned as a result of importing a dangling index.
`NEW_INDEX_RESTORED`:: Unassigned as a result of restoring into a new index.
`EXISTING_INDEX_RESTORED`:: Unassigned as a result of restoring into a closed index.
`REPLICA_ADDED`:: Unassigned as a result of explicit addition of a replica.
`ALLOCATION_FAILED`:: Unassigned as a result of a failed allocation of the shard.
`NODE_LEFT`:: Unassigned as a result of the node hosting it leaving the cluster.
`REROUTE_CANCELLED`:: Unassigned as a result of explicit cancel reroute command.
`REINITIALIZED`:: When a shard moves from started back to initializing, for example, with shadow replicas.
`REALLOCATED_REPLICA`:: A better replica location is identified and causes the existing replica allocation to be cancelled.

View File

@ -132,7 +132,8 @@ the operating system:
Recent CPU usage for the whole system, or -1 if not supported
`os.cpu.load_average`::
System load average for the last minute, or -1 if not supported
Array of system load averages for the last one minute, five
minute and fifteen minutes (value of -1 indicates not supported)
`os.mem.total_in_bytes`::
Total amount of physical memory in bytes

View File

@ -250,3 +250,36 @@ PUT my_index/my_type/1
<1> The `english` field is mapped as a `string` field with the `english` analyzer.
<2> The `count` field is mapped as a `long` field with `doc_values` disabled
[[override-default-template]]
=== Override default template
You can override the default mappings for all indices and all types
by specifying a `_default_` type mapping in an index template
which matches all indices.
For example, to disable the `_all` field by default for all types in all
new indices, you could create the following index template:
[source,js]
--------------------------------------------------
PUT _template/disable_all_field
{
"disable_all_field": {
"order": 0,
"template": "*", <1>
"mappings": {
"_default_": { <2>
"_all": { <3>
"enabled": false
}
}
}
}
}
--------------------------------------------------
// AUTOSENSE
<1> Applies the mappings to an `index` which matches the pattern `*`, in other
words, all new indices.
<2> Defines the `_default_` type mapping types within the index.
<3> Disables the `_all` field by default.

View File

@ -32,7 +32,7 @@ In addition, the following routing-related node settings have been deprecated:
The delete API used to be broadcast to all shards in the index which meant
that, when using custom routing, the `routing` parameter was optional. Now,
the delete request is forwarded only to the document holding the shard. If you
the delete request is forwarded only to the shard holding the document. If you
are using custom routing then you should specify the `routing` value when
deleting a document, just as is already required for the `index`, `create`,
and `update` APIs.

View File

@ -194,7 +194,6 @@ Previously, there were three settings for the ping timeout: `discovery.zen.initi
the only setting key for the ping timeout is now `discovery.zen.ping_timeout`. The default value for
ping timeouts remains at three seconds.
==== Recovery settings
Recovery settings deprecated in 1.x have been removed:
@ -238,6 +237,11 @@ Please change the setting in your configuration files or in the clusterstate to
The 'default' similarity has been renamed to 'classic'.
==== Indexing settings
`indices.memory.min_shard_index_buffer_size` and `indices.memory.max_shard_index_buffer_size` are removed since Elasticsearch now allows any one shard to any
amount of heap as long as the total indexing buffer heap used across all shards is below the node's `indices.memory.index_buffer_size` (default: 10% of the JVM heap)
[[breaking_30_mapping_changes]]
=== Mapping changes
@ -552,17 +556,32 @@ and high risk of being misused. The ability to change the thread pool type for a
that it is still possible to adjust relevant thread pool parameters for each of the thread pools (e.g., depending on
the thread pool type, `keep_alive`, `queue_size`, etc.).
=== Adding system CPU percent to OS stats
=== System CPU stats
The recent CPU usage (as a percent) has been added to the OS stats reported under the node stats API and the cat nodes
API. The breaking change here is that there is a new object in the "os" object in the node stats response. This object
is called "cpu" and includes "percent" and "load_average" as fields. This moves the "load_average" field that was
previously a top-level field in the "os" object to the "cpu" object. Additionally, the "cpu" field in the cat nodes API
response is output by default.
The recent CPU usage (as a percent) has been added to the OS stats
reported under the node stats API and the cat nodes API. The breaking
change here is that there is a new object in the "os" object in the node
stats response. This object is called "cpu" and includes "percent" and
"load_average" as fields. This moves the "load_average" field that was
previously a top-level field in the "os" object to the "cpu" object. The
format of the "load_average" field has changed to an array of length
three representing the one-minute, five-minute and fifteen-minute load
averages (a value of -1 for any of array components indicates that the
corresponding metric is not available).
Finally, the API for org.elasticsearch.monitor.os.OsStats has changed. The `getLoadAverage` method has been removed. The
value for this can now be obtained from `OsStats.Cpu#getLoadAverage`. Additionally, the recent CPU usage can be obtained
from `OsStats.Cpu#getPercent`.
In the cat nodes API response, the "cpu" field is output by default. The
previous "load" field has been removed and is replaced by "load_1m",
"load_5m", and "load_15m" which represent the one-minute, five-minute
and fifteen-minute loads respectively. These values are output by
default, and a value of -1 indicates that the corresponding metric is
not available.
Finally, the API for org.elasticsearch.monitor.os.OsStats has
changed. The `getLoadAverage` method has been removed. The value for
this can now be obtained from `OsStats.Cpu#getLoadAverage` but it is no
longer a double and is instead an object encapuslating the one-minute,
five-minute and fifteen-minute load averages. Additionally, the recent
CPU usage can be obtained from `OsStats.Cpu#getPercent`.
=== Fields option
Only stored fields are retrievable with this option.

View File

@ -12,7 +12,7 @@ in the cluster:
Accepts either a percentage or a byte size value. It defaults to `10%`,
meaning that `10%` of the total heap allocated to a node will be used as the
indexing buffer size.
indexing buffer size shared across all shards.
`indices.memory.min_index_buffer_size`::
@ -23,10 +23,3 @@ in the cluster:
If the `index_buffer_size` is specified as a percentage, then this
setting can be used to specify an absolute maximum. Defaults to unbounded.
`indices.memory.min_shard_index_buffer_size`::
Sets a hard lower limit for the memory allocated per shard for its own
indexing buffer. Defaults to `4mb`.

View File

@ -26,8 +26,8 @@ dependencies {
compile 'org.codehaus.groovy:groovy:2.4.4:indy'
}
compileJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked,-cast,-deprecation'
compileTestJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked,-cast,-deprecation'
compileJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked,-cast'
compileTestJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked,-cast'
integTest {
cluster {

View File

@ -30,6 +30,3 @@ dependencies {
dependencyLicenses {
mapping from: /lucene-.*/, to: 'lucene'
}
compileJava.options.compilerArgs << "-Xlint:-deprecation"

View File

@ -19,18 +19,19 @@
package org.elasticsearch.index.analysis;
import com.ibm.icu.text.Collator;
import com.ibm.icu.text.RuleBasedCollator;
import com.ibm.icu.util.ULocale;
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.file.Files;
import org.apache.lucene.analysis.TokenStream;
import org.elasticsearch.common.io.Streams;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.file.Files;
import com.ibm.icu.text.Collator;
import com.ibm.icu.text.RuleBasedCollator;
import com.ibm.icu.util.ULocale;
/**
* An ICU based collation token filter. There are two ways to configure collation:
@ -45,6 +46,7 @@ public class IcuCollationTokenFilterFactory extends AbstractTokenFilterFactory {
private final Collator collator;
@SuppressWarnings("deprecation") // Intentionally sets deprecated options for backwards compatibility
public IcuCollationTokenFilterFactory(IndexSettings indexSettings, Environment environment, String name, Settings settings) {
super(indexSettings, name, settings);
@ -165,6 +167,7 @@ public class IcuCollationTokenFilterFactory extends AbstractTokenFilterFactory {
}
@Override
@SuppressWarnings("deprecation") // Constructs a deprecated filter for backwards compatibility
public TokenStream create(TokenStream tokenStream) {
return new ICUCollationKeyFilter(tokenStream, collator);
}

View File

@ -57,8 +57,6 @@ dependencyLicenses {
}
compileJava.options.compilerArgs << '-Xlint:-path,-serial,-unchecked'
// TODO: why is deprecation needed here but not in maven....?
compileJava.options.compilerArgs << '-Xlint:-deprecation'
thirdPartyAudit.excludes = [
// classes are missing

View File

@ -42,8 +42,6 @@ dependencyLicenses {
mapping from: /jackson-.*/, to: 'jackson'
}
compileJava.options.compilerArgs << '-Xlint:-rawtypes,-deprecation'
test {
// this is needed for insecure plugins, remove if possible!
systemProperty 'tests.artifact', project.name

View File

@ -20,6 +20,7 @@
package org.elasticsearch.cloud.aws;
import com.amazonaws.services.ec2.AmazonEC2;
import org.elasticsearch.common.component.LifecycleComponent;
public interface AwsEc2Service extends LifecycleComponent<AwsEc2Service> {
@ -33,10 +34,6 @@ public interface AwsEc2Service extends LifecycleComponent<AwsEc2Service> {
public static final String PROXY_PASSWORD = "cloud.aws.proxy.password";
public static final String SIGNER = "cloud.aws.signer";
public static final String REGION = "cloud.aws.region";
@Deprecated
public static final String DEPRECATED_PROXY_HOST = "cloud.aws.proxy_host";
@Deprecated
public static final String DEPRECATED_PROXY_PORT = "cloud.aws.proxy_port";
}
final class CLOUD_EC2 {
@ -49,10 +46,6 @@ public interface AwsEc2Service extends LifecycleComponent<AwsEc2Service> {
public static final String PROXY_PASSWORD = "cloud.aws.ec2.proxy.password";
public static final String SIGNER = "cloud.aws.ec2.signer";
public static final String ENDPOINT = "cloud.aws.ec2.endpoint";
@Deprecated
public static final String DEPRECATED_PROXY_HOST = "cloud.aws.ec2.proxy_host";
@Deprecated
public static final String DEPRECATED_PROXY_PORT = "cloud.aws.ec2.proxy_port";
}
final class DISCOVERY_EC2 {

View File

@ -33,6 +33,7 @@ import com.amazonaws.internal.StaticCredentialsProvider;
import com.amazonaws.retry.RetryPolicy;
import com.amazonaws.services.ec2.AmazonEC2;
import com.amazonaws.services.ec2.AmazonEC2Client;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cloud.aws.network.Ec2NameResolver;
import org.elasticsearch.cloud.aws.node.Ec2CustomNodeAttributes;
@ -71,6 +72,7 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent<AwsEc2Service>
discoveryNodeService.addCustomAttributeProvider(new Ec2CustomNodeAttributes(settings));
}
@Override
public synchronized AmazonEC2 client() {
if (client != null) {
return client;
@ -91,11 +93,11 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent<AwsEc2Service>
String account = settings.get(CLOUD_EC2.KEY, settings.get(CLOUD_AWS.KEY));
String key = settings.get(CLOUD_EC2.SECRET, settings.get(CLOUD_AWS.SECRET));
String proxyHost = settings.get(CLOUD_AWS.PROXY_HOST, settings.get(CLOUD_AWS.DEPRECATED_PROXY_HOST));
proxyHost = settings.get(CLOUD_EC2.PROXY_HOST, settings.get(CLOUD_EC2.DEPRECATED_PROXY_HOST, proxyHost));
String proxyHost = settings.get(CLOUD_AWS.PROXY_HOST);
proxyHost = settings.get(CLOUD_EC2.PROXY_HOST, proxyHost);
if (proxyHost != null) {
String portString = settings.get(CLOUD_AWS.PROXY_PORT, settings.get(CLOUD_AWS.DEPRECATED_PROXY_PORT, "80"));
portString = settings.get(CLOUD_EC2.PROXY_PORT, settings.get(CLOUD_EC2.DEPRECATED_PROXY_PORT, portString));
String portString = settings.get(CLOUD_AWS.PROXY_PORT, "80");
portString = settings.get(CLOUD_EC2.PROXY_PORT, portString);
Integer proxyPort;
try {
proxyPort = Integer.parseInt(portString, 10);
@ -135,7 +137,7 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent<AwsEc2Service>
int retriesAttempted) {
// with 10 retries the max delay time is 320s/320000ms (10 * 2^5 * 1 * 1000)
logger.warn("EC2 API request failed, retry again. Reason was:", exception);
return 1000L * (long) (10d * Math.pow(2, ((double) retriesAttempted) / 2.0d) * (1.0d + rand.nextDouble()));
return 1000L * (long) (10d * Math.pow(2, retriesAttempted / 2.0d) * (1.0d + rand.nextDouble()));
}
},
10,

View File

@ -19,6 +19,11 @@
package org.elasticsearch.plugin.discovery.ec2;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.Collection;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.cloud.aws.AwsEc2ServiceImpl;
import org.elasticsearch.cloud.aws.Ec2Module;
@ -32,16 +37,11 @@ import org.elasticsearch.discovery.ec2.AwsEc2UnicastHostsProvider;
import org.elasticsearch.discovery.ec2.Ec2Discovery;
import org.elasticsearch.plugins.Plugin;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.Collection;
/**
*
*/
public class Ec2DiscoveryPlugin extends Plugin {
// ClientConfiguration clinit has some classloader problems
// TODO: fix that
static {
@ -87,6 +87,7 @@ public class Ec2DiscoveryPlugin extends Plugin {
}
@Override
@SuppressWarnings("rawtypes") // Supertype uses rawtype
public Collection<Class<? extends LifecycleComponent>> nodeServices() {
Collection<Class<? extends LifecycleComponent>> services = new ArrayList<>();
services.add(AwsEc2ServiceImpl.class);

View File

@ -21,5 +21,3 @@ esplugin {
description 'The Multicast Discovery plugin allows discovery other nodes using multicast requests'
classname 'org.elasticsearch.plugin.discovery.multicast.MulticastDiscoveryPlugin'
}
compileJava.options.compilerArgs << "-Xlint:-deprecation"

View File

@ -19,7 +19,18 @@
package org.elasticsearch.plugin.discovery.multicast;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import java.io.IOException;
import java.net.InetAddress;
import java.net.SocketAddress;
import java.security.AccessController;
import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.Constants;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.SpecialPermission;
@ -55,17 +66,7 @@ import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.net.InetAddress;
import java.net.SocketAddress;
import java.security.AccessController;
import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import static org.elasticsearch.cluster.node.DiscoveryNode.readNode;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
@ -144,13 +145,9 @@ public class MulticastZenPing extends AbstractLifecycleComponent<ZenPing> implem
boolean shared = settings.getAsBoolean("discovery.zen.ping.multicast.shared", Constants.MAC_OS_X);
// OSX does not correctly send multicasts FROM the right interface
boolean deferToInterface = settings.getAsBoolean("discovery.zen.ping.multicast.defer_group_to_set_interface", Constants.MAC_OS_X);
// don't use publish address, the use case for that is e.g. a firewall or proxy and
// may not even be bound to an interface on this machine! use the first bound address.
List<InetAddress> addresses = Arrays.asList(networkService.resolveBindHostAddresses(address == null ? null : new String[] { address }));
NetworkUtils.sortAddresses(addresses);
final MulticastChannel.Config config = new MulticastChannel.Config(port, group, bufferSize, ttl,
addresses.get(0), deferToInterface);
getMulticastInterface(), deferToInterface);
SecurityManager sm = System.getSecurityManager();
if (sm != null) {
sm.checkPermission(new SpecialPermission());
@ -167,6 +164,16 @@ public class MulticastZenPing extends AbstractLifecycleComponent<ZenPing> implem
}
}
@SuppressWarnings("deprecation") // Used to support funky configuration options
private InetAddress getMulticastInterface() throws IOException {
// don't use publish address, the use case for that is e.g. a firewall or proxy and
// may not even be bound to an interface on this machine! use the first bound address.
List<InetAddress> addresses = Arrays.asList(networkService.resolveBindHostAddresses(address == null ? null : new String[] { address }));
NetworkUtils.sortAddresses(addresses);
return addresses.get(0);
}
@Override
protected void doStop() {
if (multicastChannel != null) {

View File

@ -61,7 +61,7 @@ dependencies {
compile 'org.apache.commons:commons-compress:1.10'
}
compileJava.options.compilerArgs << '-Xlint:-cast,-deprecation,-rawtypes'
compileJava.options.compilerArgs << '-Xlint:-cast,-rawtypes'
forbiddenPatterns {
exclude '**/*.docx'

View File

@ -100,6 +100,7 @@ public class AttachmentMapper extends FieldMapper {
super(ref);
}
@Override
public AttachmentMapper.AttachmentFieldType clone() {
return new AttachmentMapper.AttachmentFieldType(this);
}
@ -109,6 +110,7 @@ public class AttachmentMapper extends FieldMapper {
return CONTENT_TYPE;
}
@Override
public String value(Object value) {
return value == null?null:value.toString();
}
@ -292,7 +294,7 @@ public class AttachmentMapper extends FieldMapper {
type = "string";
}
Mapper.TypeParser typeParser = parserContext.typeParser(type);
Mapper.Builder<?, ?> mapperBuilder = typeParser.parse(propName, (Map<String, Object>) propNode, parserContext);
Mapper.Builder<?, ?> mapperBuilder = typeParser.parse(propName, propNode, parserContext);
return mapperBuilder;
}
@ -414,6 +416,7 @@ public class AttachmentMapper extends FieldMapper {
}
@Override
@SuppressWarnings("deprecation") // https://github.com/elastic/elasticsearch/issues/15843
public Mapper parse(ParseContext context) throws IOException {
byte[] content = null;
String contentType = null;

View File

@ -35,6 +35,5 @@ dependencyLicenses {
mapping from: /stax-.*/, to: 'stax'
}
compileJava.options.compilerArgs << '-Xlint:-deprecation,-serial'
compileTestJava.options.compilerArgs << '-Xlint:-deprecation'
compileJava.options.compilerArgs << '-Xlint:-serial'

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cloud.azure.storage;
import com.microsoft.azure.storage.LocationMode;
import com.microsoft.azure.storage.StorageException;
import org.elasticsearch.common.blobstore.BlobMetaData;
import java.io.InputStream;
@ -36,10 +37,6 @@ public interface AzureStorageService {
final class Storage {
public static final String PREFIX = "cloud.azure.storage.";
@Deprecated
public static final String ACCOUNT_DEPRECATED = "cloud.azure.storage.account";
@Deprecated
public static final String KEY_DEPRECATED = "cloud.azure.storage.key";
public static final String TIMEOUT = "cloud.azure.storage.timeout";

View File

@ -82,52 +82,42 @@ public class AzureStorageSettings {
AzureStorageSettings primaryStorage = null;
Map<String, AzureStorageSettings> secondaryStorage = new HashMap<>();
// We check for deprecated settings
String account = settings.get(Storage.ACCOUNT_DEPRECATED);
String key = settings.get(Storage.KEY_DEPRECATED);
TimeValue globalTimeout = settings.getAsTime(Storage.TIMEOUT, TimeValue.timeValueMinutes(5));
if (account != null) {
logger.warn("[{}] and [{}] have been deprecated. Use now [{}xxx.account] and [{}xxx.key] where xxx is any name",
Storage.ACCOUNT_DEPRECATED, Storage.KEY_DEPRECATED, Storage.PREFIX, Storage.PREFIX);
primaryStorage = new AzureStorageSettings(null, account, key, globalTimeout);
} else {
Settings storageSettings = settings.getByPrefix(Storage.PREFIX);
if (storageSettings != null) {
Map<String, Object> asMap = storageSettings.getAsStructuredMap();
for (Map.Entry<String, Object> storage : asMap.entrySet()) {
if (storage.getValue() instanceof Map) {
@SuppressWarnings("unchecked")
Map<String, String> map = (Map) storage.getValue();
TimeValue timeout = TimeValue.parseTimeValue(map.get("timeout"), globalTimeout, Storage.PREFIX + storage.getKey() + ".timeout");
AzureStorageSettings current = new AzureStorageSettings(storage.getKey(), map.get("account"), map.get("key"), timeout);
boolean activeByDefault = Boolean.parseBoolean(map.getOrDefault("default", "false"));
if (activeByDefault) {
if (primaryStorage == null) {
primaryStorage = current;
} else {
logger.warn("default storage settings has already been defined. You can not define it to [{}]", storage.getKey());
secondaryStorage.put(storage.getKey(), current);
}
Settings storageSettings = settings.getByPrefix(Storage.PREFIX);
if (storageSettings != null) {
Map<String, Object> asMap = storageSettings.getAsStructuredMap();
for (Map.Entry<String, Object> storage : asMap.entrySet()) {
if (storage.getValue() instanceof Map) {
@SuppressWarnings("unchecked")
Map<String, String> map = (Map) storage.getValue();
TimeValue timeout = TimeValue.parseTimeValue(map.get("timeout"), globalTimeout, Storage.PREFIX + storage.getKey() + ".timeout");
AzureStorageSettings current = new AzureStorageSettings(storage.getKey(), map.get("account"), map.get("key"), timeout);
boolean activeByDefault = Boolean.parseBoolean(map.getOrDefault("default", "false"));
if (activeByDefault) {
if (primaryStorage == null) {
primaryStorage = current;
} else {
logger.warn("default storage settings has already been defined. You can not define it to [{}]", storage.getKey());
secondaryStorage.put(storage.getKey(), current);
}
} else {
secondaryStorage.put(storage.getKey(), current);
}
}
// If we did not set any default storage, we should complain and define it
if (primaryStorage == null && secondaryStorage.isEmpty() == false) {
Map.Entry<String, AzureStorageSettings> fallback = secondaryStorage.entrySet().iterator().next();
// We only warn if the number of secondary storage if > to 1
// If the user defined only one storage account, that's fine. We know it's the default one.
if (secondaryStorage.size() > 1) {
logger.warn("no default storage settings has been defined. " +
"Add \"default\": true to the settings you want to activate by default. " +
"Forcing default to [{}].", fallback.getKey());
}
primaryStorage = fallback.getValue();
secondaryStorage.remove(fallback.getKey());
}
// If we did not set any default storage, we should complain and define it
if (primaryStorage == null && secondaryStorage.isEmpty() == false) {
Map.Entry<String, AzureStorageSettings> fallback = secondaryStorage.entrySet().iterator().next();
// We only warn if the number of secondary storage if > to 1
// If the user defined only one storage account, that's fine. We know it's the default one.
if (secondaryStorage.size() > 1) {
logger.warn("no default storage settings has been defined. " +
"Add \"default\": true to the settings you want to activate by default. " +
"Forcing default to [{}].", fallback.getKey());
}
primaryStorage = fallback.getValue();
secondaryStorage.remove(fallback.getKey());
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cloud.azure;
import com.microsoft.azure.storage.LocationMode;
import com.microsoft.azure.storage.StorageException;
import org.elasticsearch.cloud.azure.storage.AzureStorageService;
import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
import org.elasticsearch.cloud.azure.storage.AzureStorageServiceMock;
@ -79,11 +80,6 @@ public abstract class AbstractAzureRepositoryServiceTestCase extends AbstractAzu
protected Settings nodeSettings(int nodeOrdinal) {
Settings.Builder builder = Settings.settingsBuilder()
.put(Storage.CONTAINER, "snapshots");
// We use sometime deprecated settings in tests
builder.put(Storage.ACCOUNT_DEPRECATED, "mock_azure_account")
.put(Storage.KEY_DEPRECATED, "mock_azure_key");
return builder.build();
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.repositories.azure;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
import org.elasticsearch.cloud.azure.storage.AzureStorageSettings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.Settings;
@ -66,19 +65,6 @@ public class AzureSettingsParserTest extends LuceneTestCase {
assertThat(tuple.v2().keySet(), hasSize(0));
}
public void testDeprecatedSettings() {
Settings settings = Settings.builder()
.put(Storage.ACCOUNT_DEPRECATED, "myaccount1")
.put(Storage.KEY_DEPRECATED, "mykey1")
.build();
Tuple<AzureStorageSettings, Map<String, AzureStorageSettings>> tuple = AzureStorageSettings.parse(settings);
assertThat(tuple.v1(), notNullValue());
assertThat(tuple.v1().getAccount(), is("myaccount1"));
assertThat(tuple.v1().getKey(), is("mykey1"));
assertThat(tuple.v2().keySet(), hasSize(0));
}
public void testParseTwoSettingsNoDefault() {
Settings settings = Settings.builder()
.put("cloud.azure.storage.azure1.account", "myaccount1")

View File

@ -96,8 +96,6 @@ integTest {
}
}
compileJava.options.compilerArgs << '-Xlint:-deprecation,-rawtypes'
thirdPartyAudit.excludes = [
// classes are missing, because we added hadoop jars one by one until tests pass.
'com.google.gson.stream.JsonReader',

View File

@ -43,8 +43,6 @@ dependencyLicenses {
mapping from: /jackson-.*/, to: 'jackson'
}
compileJava.options.compilerArgs << '-Xlint:-deprecation,-rawtypes'
test {
// this is needed for insecure plugins, remove if possible!
systemProperty 'tests.artifact', project.name

View File

@ -20,6 +20,7 @@
package org.elasticsearch.cloud.aws;
import com.amazonaws.services.s3.AmazonS3;
import org.elasticsearch.common.component.LifecycleComponent;
/**
@ -37,10 +38,6 @@ public interface AwsS3Service extends LifecycleComponent<AwsS3Service> {
public static final String PROXY_PASSWORD = "cloud.aws.proxy.password";
public static final String SIGNER = "cloud.aws.signer";
public static final String REGION = "cloud.aws.region";
@Deprecated
public static final String DEPRECATED_PROXY_HOST = "cloud.aws.proxy_host";
@Deprecated
public static final String DEPRECATED_PROXY_PORT = "cloud.aws.proxy_port";
}
final class CLOUD_S3 {
@ -53,10 +50,6 @@ public interface AwsS3Service extends LifecycleComponent<AwsS3Service> {
public static final String PROXY_PASSWORD = "cloud.aws.s3.proxy.password";
public static final String SIGNER = "cloud.aws.s3.signer";
public static final String ENDPOINT = "cloud.aws.s3.endpoint";
@Deprecated
public static final String DEPRECATED_PROXY_HOST = "cloud.aws.s3.proxy_host";
@Deprecated
public static final String DEPRECATED_PROXY_PORT = "cloud.aws.s3.proxy_port";
}
final class REPOSITORY_S3 {

View File

@ -31,6 +31,7 @@ import com.amazonaws.http.IdleConnectionReaper;
import com.amazonaws.internal.StaticCredentialsProvider;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
@ -94,7 +95,6 @@ public class InternalAwsS3Service extends AbstractLifecycleComponent<AwsS3Servic
return getClient(endpoint, protocol, account, key, maxRetries);
}
private synchronized AmazonS3 getClient(String endpoint, String protocol, String account, String key, Integer maxRetries) {
Tuple<String, String> clientDescriptor = new Tuple<String, String>(endpoint, account);
AmazonS3Client client = clients.get(clientDescriptor);
@ -119,11 +119,11 @@ public class InternalAwsS3Service extends AbstractLifecycleComponent<AwsS3Servic
throw new IllegalArgumentException("No protocol supported [" + protocol + "], can either be [http] or [https]");
}
String proxyHost = settings.get(CLOUD_AWS.PROXY_HOST, settings.get(CLOUD_AWS.DEPRECATED_PROXY_HOST));
proxyHost = settings.get(CLOUD_S3.PROXY_HOST, settings.get(CLOUD_S3.DEPRECATED_PROXY_HOST, proxyHost));
String proxyHost = settings.get(CLOUD_AWS.PROXY_HOST);
proxyHost = settings.get(CLOUD_S3.PROXY_HOST, proxyHost);
if (proxyHost != null) {
String portString = settings.get(CLOUD_AWS.PROXY_PORT, settings.get(CLOUD_AWS.DEPRECATED_PROXY_PORT, "80"));
portString = settings.get(CLOUD_S3.PROXY_PORT, settings.get(CLOUD_S3.DEPRECATED_PROXY_PORT, portString));
String portString = settings.get(CLOUD_AWS.PROXY_PORT, "80");
portString = settings.get(CLOUD_S3.PROXY_PORT, portString);
Integer proxyPort;
try {
proxyPort = Integer.parseInt(portString, 10);

View File

@ -77,6 +77,7 @@ public class S3RepositoryPlugin extends Plugin {
}
@Override
@SuppressWarnings("rawtypes") // Supertype declaration has raw types
public Collection<Class<? extends LifecycleComponent>> nodeServices() {
return Collections.<Class<? extends LifecycleComponent>>singleton(S3Module.getS3ServiceImpl());
}

View File

@ -6,8 +6,8 @@
- match:
$body: |
/ #host ip heap.percent ram.percent cpu load node.role master name
^ (\S+ \s+ (\d{1,3}\.){3}\d{1,3} \s+ \d+ \s+ \d* \s+ (-)?\d* \s+ (-)?\d*(\.\d+)? \s+ [-dc] \s+ [-*mx] \s+ (\S+\s?)+ \n)+ $/
/ #host ip heap.percent ram.percent cpu load_1m load_5m load_15m node.role master name
^ (\S+ \s+ (\d{1,3}\.){3}\d{1,3} \s+ \d+ \s+ \d* \s+ (-)?\d* \s+ ((-)?\d*(\.\d+)?)? \s+ ((-)?\d*(\.\d+)?)? \s+ ((-)?\d*(\.\d+)?)? \s+ [-dc] \s+ [-*mx] \s+ (\S+\s?)+ \n)+ $/
- do:
cat.nodes:
@ -15,8 +15,8 @@
- match:
$body: |
/^ host \s+ ip \s+ heap\.percent \s+ ram\.percent \s+ cpu \s+ load \s+ node\.role \s+ master \s+ name \n
(\S+ \s+ (\d{1,3}\.){3}\d{1,3} \s+ \d+ \s+ \d* \s+ (-)?\d* \s+ (-)?\d*(\.\d+)? \s+ [-dc] \s+ [-*mx] \s+ (\S+\s?)+ \n)+ $/
/^ host \s+ ip \s+ heap\.percent \s+ ram\.percent \s+ cpu \s+ load_1m \s+ load_5m \s+ load_15m \s+ node\.role \s+ master \s+ name \n
(\S+ \s+ (\d{1,3}\.){3}\d{1,3} \s+ \d+ \s+ \d* \s+ (-)?\d* \s+ ((-)?\d*(\.\d+)?)? \s+ ((-)?\d*(\.\d+)?)? \s+ ((-)?\d*(\.\d+)?)? \s+ [-dc] \s+ [-*mx] \s+ (\S+\s?)+ \n)+ $/
- do:
cat.nodes:

View File

@ -33,7 +33,7 @@ dependencies {
compile 'org.elasticsearch:securemock:1.2'
}
compileJava.options.compilerArgs << '-Xlint:-cast,-deprecation,-rawtypes,-serial,-try,-unchecked'
compileJava.options.compilerArgs << '-Xlint:-cast,-rawtypes,-serial,-try,-unchecked'
compileTestJava.options.compilerArgs << '-Xlint:-rawtypes'
// the main files are actually test files, so use the appopriate forbidden api sigs

View File

@ -23,21 +23,22 @@ import org.apache.lucene.util.LuceneTestCase;
import java.nio.file.FileSystem;
/**
* Exposes some package private stuff in PathUtils for framework purposes only!
/**
* Exposes some package private stuff in PathUtils for framework purposes only!
*/
public class PathUtilsForTesting {
/** Sets a new default filesystem for testing */
@SuppressWarnings("deprecation") // https://github.com/elastic/elasticsearch/issues/15845
public static void setup() {
installMock(LuceneTestCase.getBaseTempDirForTestClass().getFileSystem());
}
/** Installs a mock filesystem for testing */
public static void installMock(FileSystem mock) {
PathUtils.DEFAULT = mock;
}
/** Resets filesystem back to the real system default */
public static void teardown() {
PathUtils.DEFAULT = PathUtils.ACTUAL_DEFAULT;

View File

@ -24,6 +24,7 @@ import com.carrotsearch.randomizedtesting.SysGlobals;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import org.apache.lucene.store.StoreRateLimiting;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
@ -117,7 +118,6 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static junit.framework.Assert.fail;
import static org.apache.lucene.util.LuceneTestCase.TEST_NIGHTLY;
import static org.apache.lucene.util.LuceneTestCase.rarely;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -127,6 +127,7 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
/**
* InternalTestCluster manages a set of JVM private nodes and allows convenient access to them.
@ -1045,6 +1046,7 @@ public final class InternalTestCluster extends TestCluster {
}
}
@SuppressWarnings("deprecation") // https://github.com/elastic/elasticsearch/issues/15844
private void randomlyResetClients() throws IOException {
// only reset the clients on nightly tests, it causes heavy load...
if (RandomizedTest.isNightly() && rarely(random)) {

View File

@ -75,6 +75,7 @@ public class LongGCDisruption extends SingleNodeDisruption {
return TimeValue.timeValueMillis(0);
}
@SuppressWarnings("deprecation") // stops/resumes threads intentionally
@SuppressForbidden(reason = "stops/resumes threads intentionally")
protected boolean stopNodeThreads(String node, Set<Thread> nodeThreads) {
Thread[] allThreads = null;
@ -118,6 +119,7 @@ public class LongGCDisruption extends SingleNodeDisruption {
return stopped;
}
@SuppressWarnings("deprecation") // stops/resumes threads intentionally
@SuppressForbidden(reason = "stops/resumes threads intentionally")
protected void resumeThreads(Set<Thread> threads) {
for (Thread thread : threads) {

View File

@ -513,9 +513,9 @@ public class ElasticsearchAssertions {
public static <T extends Query> T assertBooleanSubQuery(Query query, Class<T> subqueryType, int i) {
assertThat(query, instanceOf(BooleanQuery.class));
BooleanQuery q = (BooleanQuery) query;
assertThat(q.getClauses().length, greaterThan(i));
assertThat(q.getClauses()[i].getQuery(), instanceOf(subqueryType));
return (T) q.getClauses()[i].getQuery();
assertThat(q.clauses().size(), greaterThan(i));
assertThat(q.clauses().get(i).getQuery(), instanceOf(subqueryType));
return subqueryType.cast(q.clauses().get(i).getQuery());
}
/**

View File

@ -21,6 +21,7 @@ package org.elasticsearch.test.store;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.store.BaseDirectoryWrapper;
@ -106,6 +107,7 @@ public class MockFSDirectoryService extends FsDirectoryService {
throw new UnsupportedOperationException();
}
@SuppressWarnings("deprecation") // https://github.com/elastic/elasticsearch/issues/15846
public static void checkIndex(ESLogger logger, Store store, ShardId shardId) {
if (store.tryIncRef()) {
logger.info("start check index");