From b3357f09feb228c0f9ccc7dc75cadce803337b87 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 14 Oct 2015 05:41:41 -0400 Subject: [PATCH 01/33] a start --- .../elasticsearch/cluster/ClusterModule.java | 3 +- .../metadata/MetaDataIndexUpgradeService.java | 3 +- .../elasticsearch/index/engine/Engine.java | 6 +- .../index/engine/EngineConfig.java | 67 +--- .../index/engine/InternalEngine.java | 45 +-- .../index/engine/ShadowEngine.java | 4 +- .../elasticsearch/index/shard/IndexShard.java | 112 ++----- .../translog/BufferingTranslogWriter.java | 12 - .../index/translog/Translog.java | 9 +- .../index/translog/TranslogConfig.java | 15 +- .../index/translog/TranslogWriter.java | 2 +- .../memory/IndexingMemoryController.java | 307 ++++-------------- .../engine/InternalEngineSettingsTests.java | 46 --- .../index/engine/InternalEngineTests.java | 15 +- .../index/shard/IndexShardTests.java | 2 - .../memory/IndexingMemoryControllerIT.java | 104 ------ .../memory/IndexingMemoryControllerTests.java | 183 +++-------- 17 files changed, 174 insertions(+), 761 deletions(-) delete mode 100644 core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerIT.java diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java index 581bee10369..df1529a1957 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -231,7 +231,6 @@ public class ClusterModule extends AbstractModule { registerIndexDynamicSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH, Validator.BOOLEAN); 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); @@ -324,4 +323,4 @@ public class ClusterModule extends AbstractModule { bind(NodeMappingRefreshAction.class).asEagerSingleton(); bind(MappingUpdatedAction.class).asEagerSingleton(); } -} \ No newline at end of file +} diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index cdde49170d4..29a3dcaeb9f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -125,7 +125,8 @@ public class MetaDataIndexUpgradeService extends AbstractComponent { "index.store.throttle.max_bytes_per_sec", "index.translog.flush_threshold_size", "index.translog.fs.buffer_size", - "index.version_map_size")); + "index.version_map_size", + "index.buffer_size")); /** All known time settings for an index. */ public static final Set INDEX_TIME_SETTINGS = unmodifiableSet(newHashSet( diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index c07be064489..90640f0695e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -361,8 +361,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 */ + abstract public long indexBufferRAMBytesUsed(); protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { ensureOpen(); @@ -460,7 +460,7 @@ 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; diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index fd4b5daf4ee..e76aa1d38b9 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -54,9 +54,7 @@ public final class EngineConfig { private final ShardId shardId; private final TranslogRecoveryPerformer translogRecoveryPerformer; private final Settings indexSettings; - private volatile ByteSizeValue indexingBufferSize; - private volatile ByteSizeValue versionMapSize; - private volatile String versionMapSizeSetting; + private final ByteSizeValue indexingBufferSize; private volatile boolean compoundOnFlush = true; private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis(); private volatile boolean enableGcDeletes = true; @@ -96,21 +94,17 @@ 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%) + * Index setting to control the index buffer size. + * This setting is not realtime updateable. */ - public static final String INDEX_VERSION_MAP_SIZE = "index.version_map_size"; - + public static final String INDEX_BUFFER_SIZE_SETTING = "index.buffer_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; @@ -136,13 +130,12 @@ public final class EngineConfig { this.similarity = similarity; this.codecService = codecService; this.failedEngineListener = failedEngineListener; - this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); - codecName = indexSettings.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; - gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis(); - versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE); - updateVersionMapSize(); + this.compoundOnFlush = indexSettings.getAsBoolean(INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); + codecName = indexSettings.get(INDEX_CODEC_SETTING, DEFAULT_CODEC_NAME); + // We tell IndexWriter to use large heap, but IndexingMemoryController checks periodically and refreshes the most heap-consuming + // shards when total indexing heap usage is too high: + indexingBufferSize = indexSettings.getAsBytesSize(INDEX_BUFFER_SIZE_SETTING, new ByteSizeValue(256, ByteSizeUnit.MB)); + gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, DEFAULT_GC_DELETES).millis(); this.translogRecoveryPerformer = translogRecoveryPerformer; this.forceNewTranslog = indexSettings.getAsBoolean(INDEX_FORCE_NEW_TRANSLOG, false); this.queryCache = queryCache; @@ -150,51 +143,11 @@ public final class EngineConfig { this.translogConfig = translogConfig; } - /** 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 * diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3973b47f3ac..f264af2a0aa 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -101,6 +101,8 @@ public class InternalEngine extends Engine { private volatile SegmentInfos lastCommittedSegmentInfos; + private volatile boolean refreshing; + private final IndexThrottle throttle; public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogRecovery) throws EngineException { @@ -295,7 +297,6 @@ public class InternalEngine extends Engine { private void updateIndexWriterSettings() { try { final LiveIndexWriterConfig iwc = indexWriter.getConfig(); - iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().mbFrac()); iwc.setUseCompoundFile(engineConfig.isCompoundOnFlush()); } catch (AlreadyClosedException ex) { // ignore @@ -346,7 +347,6 @@ public class InternalEngine extends Engine { maybeFailEngine("index", t); throw new IndexFailedEngineException(shardId, index.type(), index.id(), t); } - checkVersionMapRefresh(); return created; } @@ -411,33 +411,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()) { @@ -450,7 +423,6 @@ public class InternalEngine extends Engine { } maybePruneDeletedTombstones(); - checkVersionMapRefresh(); } private void maybePruneDeletedTombstones() { @@ -516,6 +488,7 @@ public class InternalEngine extends Engine { // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); + refreshing = true; searcherManager.maybeRefreshBlocking(); } catch (AlreadyClosedException e) { ensureOpen(); @@ -525,6 +498,8 @@ public class InternalEngine extends Engine { } catch (Throwable t) { failEngine("refresh failed", t); throw new RefreshFailedEngineException(shardId, t); + } finally { + refreshing = false; } // TODO: maybe we should just put a scheduled job in threadPool? @@ -782,8 +757,12 @@ public class InternalEngine extends Engine { } @Override - public long indexWriterRAMBytesUsed() { - return indexWriter.ramBytesUsed(); + public long indexBufferRAMBytesUsed() { + if (refreshing) { + return 0; + } else { + return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); + } } @Override @@ -1098,8 +1077,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(); } diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 921f1167f43..82aee8340fd 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -234,8 +234,8 @@ public class ShadowEngine extends Engine { } @Override - public long indexWriterRAMBytesUsed() { - // No IndexWriter + public long indexBufferRAMBytesUsed() { + // No IndexWriter nor version map throw new UnsupportedOperationException("ShadowEngine has no IndexWriter"); } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 2ff8c37db3b..292ef01b8d1 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -189,11 +189,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett private final IndexSearcherWrapper searcherWrapper; - /** True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link - * IndexingMemoryController}). */ - private final AtomicBoolean active = new AtomicBoolean(); - - private volatile long lastWriteNS; private final IndexingMemoryController indexingMemoryController; @Inject @@ -253,9 +248,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett if (mapperService.hasMapping(PercolatorService.TYPE_NAME)) { percolatorQueriesRegistry.enableRealTimePercolator(); } - - // We start up inactive - active.set(false); } public Store store() { @@ -458,7 +450,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett */ public boolean index(Engine.Index index) { ensureWriteAllowed(index); - markLastWrite(index); index = indexingService.preIndex(index); final boolean created; try { @@ -483,7 +474,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett public void delete(Engine.Delete delete) { ensureWriteAllowed(delete); - markLastWrite(delete); delete = indexingService.preDelete(delete); try { if (logger.isTraceEnabled()) { @@ -893,22 +883,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett } } - /** Returns timestamp of last indexing operation */ - public long getLastWriteNS() { - return lastWriteNS; - } - - /** Records timestamp of the last write operation, possibly switching {@code active} to true if we were inactive. */ - private void markLastWrite(Engine.Operation op) { - lastWriteNS = op.startTime(); - if (active.getAndSet(true) == false) { - // We are currently inactive, but a new write operation just showed up, so we now notify IMC - // to wake up and fix our indexing buffer. We could do this async instead, but cost should - // be low, and it's rare this happens. - indexingMemoryController.forceCheck(); - } - } - private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardStateException { Engine.Operation.Origin origin = op.origin(); IndexShardState state = this.state; // one time volatile read @@ -972,70 +946,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett this.failedEngineListener.delegates.add(failedEngineListener); } - /** 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, ByteSizeValue shardTranslogBufferSize) { - - final EngineConfig config = engineConfig; - final ByteSizeValue preValue = config.getIndexingBufferSize(); - - config.setIndexingBufferSize(shardIndexingBufferSize); - + public long getIndexBufferRAMBytesUsed() { Engine engine = getEngineOrNull(); if (engine == null) { - logger.debug("updateBufferSize: engine is closed; skipping"); - return; + return 0; } - - // 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); - } - } - - engine.getTranslog().updateBuffer(shardTranslogBufferSize); - } - - /** 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(long inactiveTimeNS) { - if (System.nanoTime() - lastWriteNS >= inactiveTimeNS) { - boolean wasActive = active.getAndSet(false); - if (wasActive) { - updateBufferSize(IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER, IndexingMemoryController.INACTIVE_SHARD_TRANSLOG_BUFFER); - logger.debug("shard is now inactive"); - indicesLifecycle.onShardInactive(this); - } - } - - return active.get() == false; - } - - /** Returns {@code true} if this shard is active (has seen indexing ops in the last {@link - * IndexingMemoryController#SHARD_INACTIVE_TIME_SETTING} (default 5 minutes), else {@code false}. */ - public boolean getActive() { - return active.get(); + return engine.indexBufferRAMBytesUsed(); } public final boolean isFlushOnClose() { @@ -1163,10 +1079,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett config.setCompoundOnFlush(compoundOnFlush); 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()) { @@ -1219,6 +1131,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett return percolatorQueriesRegistry.stats(); } + /** + * Asynchronously refreshes the engine for new search operations to reflect the latest + * changes. + */ + public void refreshAsync(final String reason) { + // nocommit this really is async??? + engineConfig.getThreadPool().executor(ThreadPool.Names.REFRESH).execute(new Runnable() { + @Override + public void run() { + try { + refresh(reason); + } catch (EngineClosedException ex) { + // ignore + } + } + }); + } + class EngineRefresher implements Runnable { @Override public void run() { diff --git a/core/src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java b/core/src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java index 0e84c73f47a..05b5b917879 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java +++ b/core/src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java @@ -116,18 +116,6 @@ public final class BufferingTranslogWriter extends TranslogWriter { } } - - public void updateBufferSize(int bufferSize) { - try (ReleasableLock lock = writeLock.acquire()) { - if (this.buffer.length != bufferSize) { - flush(); - this.buffer = new byte[bufferSize]; - } - } catch (IOException e) { - throw new TranslogException(shardId, "failed to flush", e); - } - } - class WrapperOutputStream extends OutputStream { @Override diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index 4265d611fbf..64c9c456611 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -252,13 +252,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC throw new IllegalArgumentException("can't parse id from file: " + fileName); } - public void updateBuffer(ByteSizeValue bufferSize) { - config.setBufferSize(bufferSize.bytesAsInt()); - try (ReleasableLock lock = writeLock.acquire()) { - current.updateBufferSize(config.getBufferSize()); - } - } - boolean isOpen() { return closed.get() == false; } @@ -335,7 +328,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC TranslogWriter createWriter(long fileGeneration) throws IOException { TranslogWriter newFile; try { - newFile = TranslogWriter.create(config.getType(), shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), new OnCloseRunnable(), config.getBufferSize()); + newFile = TranslogWriter.create(config.getType(), shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), new OnCloseRunnable(), config.getBufferSizeBytes()); } catch (IOException e) { throw new TranslogException(shardId, "failed to create new translog file", e); } diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java index 30ab8144e1e..c831eb5aafb 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java @@ -48,7 +48,7 @@ public final class TranslogConfig { private final BigArrays bigArrays; private final ThreadPool threadPool; private final boolean syncOnEachOperation; - private volatile int bufferSize; + private final int bufferSizeBytes; private volatile TranslogGeneration translogGeneration; private volatile Translog.Durabilty durabilty = Translog.Durabilty.REQUEST; private volatile TranslogWriter.Type type; @@ -73,7 +73,7 @@ public final class TranslogConfig { this.threadPool = threadPool; this.bigArrays = bigArrays; this.type = TranslogWriter.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.BUFFERED.name())); - this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, IndexingMemoryController.INACTIVE_SHARD_TRANSLOG_BUFFER).bytes(); // Not really interesting, updated by IndexingMemoryController... + this.bufferSizeBytes = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, IndexingMemoryController.SHARD_TRANSLOG_BUFFER).bytes(); syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5)); if (syncInterval.millis() > 0 && threadPool != null) { @@ -130,15 +130,8 @@ public final class TranslogConfig { /** * Retruns the current translog buffer size. */ - public int getBufferSize() { - return bufferSize; - } - - /** - * Sets the current buffer size - for setting a live setting use {@link Translog#updateBuffer(ByteSizeValue)} - */ - public void setBufferSize(int bufferSize) { - this.bufferSize = bufferSize; + public int getBufferSizeBytes() { + return bufferSizeBytes; } /** diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index 2290dd69d87..dee4c55b06a 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -80,7 +80,7 @@ public class TranslogWriter extends TranslogReader { writeCheckpoint(headerLength, 0, file.getParent(), fileGeneration, StandardOpenOption.WRITE); final TranslogWriter writer = type.create(shardId, fileGeneration, new ChannelReference(file, fileGeneration, channel, onClose), bufferSize); return writer; - } catch (Throwable throwable){ + } catch (Throwable throwable) { IOUtils.closeWhileHandlingException(channel); try { Files.delete(file); // remove the file as well diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 90bb4c41a38..aebaeccf483 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -19,6 +19,9 @@ package org.elasticsearch.indices.memory; +import java.util.*; +import java.util.concurrent.ScheduledFuture; + import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; @@ -37,9 +40,6 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.threadpool.ThreadPool; -import java.util.*; -import java.util.concurrent.ScheduledFuture; - public class IndexingMemoryController extends AbstractLifecycleComponent { /** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */ @@ -51,51 +51,17 @@ public class IndexingMemoryController extends AbstractLifecycleComponentindices.memory.index_buffer_size 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"; + /** 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 index buffer size (default: 512 MB). */ - public static final String MAX_SHARD_INDEX_BUFFER_SIZE_SETTING = "indices.memory.max_shard_index_buffer_size"; - - /** How much heap (% or bytes) we will share across all actively indexing shards for the translog buffer (default: 1%). */ - public static final String TRANSLOG_BUFFER_SIZE_SETTING = "indices.memory.translog_buffer_size"; - - /** Only applies when indices.memory.translog_buffer_size is a %, to set a floor on the actual size in bytes (default: 256 KB). */ - public static final String MIN_TRANSLOG_BUFFER_SIZE_SETTING = "indices.memory.min_translog_buffer_size"; - - /** Only applies when indices.memory.translog_buffer_size is a %, to set a ceiling on the actual size in bytes (default: not set). */ - public static final String MAX_TRANSLOG_BUFFER_SIZE_SETTING = "indices.memory.max_translog_buffer_size"; - - /** 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"; - - /** 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"; - - /** 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"; - - /** 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"); - - /** Once a shard becomes inactive, we reduce the {@code Translog} buffer to this value (1 KB) to let active shards use the heap instead. */ - public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb", "INACTIVE_SHARD_TRANSLOG_BUFFER"); + /** Hardwired translog buffer size */ + public static final ByteSizeValue SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("32kb", "SHARD_TRANSLOG_BUFFER"); private final ThreadPool threadPool; private final IndicesService indicesService; private final ByteSizeValue indexingBuffer; - private final ByteSizeValue minShardIndexBufferSize; - private final ByteSizeValue maxShardIndexBufferSize; - private final ByteSizeValue translogBuffer; - private final ByteSizeValue minShardTranslogBufferSize; - private final ByteSizeValue maxShardTranslogBufferSize; - - private final TimeValue inactiveTime; private final TimeValue interval; private volatile ScheduledFuture scheduler; @@ -134,43 +100,13 @@ public class IndexingMemoryController extends AbstractLifecycleComponent maxTranslogBuffer.bytes()) { - translogBuffer = maxTranslogBuffer; - } - } else { - translogBuffer = ByteSizeValue.parseBytesSizeValue(translogBufferSetting, TRANSLOG_BUFFER_SIZE_SETTING); - } - this.translogBuffer = translogBuffer; - this.minShardTranslogBufferSize = this.settings.getAsBytesSize(MIN_SHARD_TRANSLOG_BUFFER_SIZE_SETTING, new ByteSizeValue(2, ByteSizeUnit.KB)); - this.maxShardTranslogBufferSize = this.settings.getAsBytesSize(MAX_SHARD_TRANSLOG_BUFFER_SIZE_SETTING, new ByteSizeValue(64, ByteSizeUnit.KB)); - - this.inactiveTime = this.settings.getAsTime(SHARD_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5)); - // 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)); + // 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_TIME_SETTING, this.inactiveTime, - SHARD_INACTIVE_INTERVAL_TIME_SETTING, this.interval); + logger.debug("using indexing buffer size [{}] with {} [{}]", this.indexingBuffer, SHARD_MEMORY_INTERVAL_TIME_SETTING, this.interval); } @Override @@ -197,15 +133,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent availableShards() { ArrayList list = new ArrayList<>(); @@ -224,6 +151,24 @@ public class IndexingMemoryController extends AbstractLifecycleComponent { + final long bytesUsed; + final ShardId shardId; - // True if the shard was active last time we checked - private final Map shardWasActive = new HashMap<>(); + public ShardAndBytesUsed(long bytesUsed, ShardId shardId) { + this.bytesUsed = bytesUsed; + this.shardId = shardId; + } + + @Override + public int compareTo(ShardAndBytesUsed other) { + // Sort larger shards first: + return Long.compare(other.bytesUsed, bytesUsed); + } + } + + class ShardsIndicesStatusChecker implements Runnable { @Override public synchronized void run() { - EnumSet changes = purgeDeletedAndClosedShards(); - updateShardStatuses(changes); - - if (changes.isEmpty() == false) { - // Something changed: recompute indexing buffers: - calcAndSetShardBuffers("[" + changes + "]"); - } - } - - /** - * goes through all existing shards and check whether there are changes in their active status - */ - private void updateShardStatuses(EnumSet changes) { + // Fast check to sum up how much heap all shards' indexing buffers are using now: + long totalBytesUsed = 0; for (ShardId shardId : availableShards()) { - - // Is the shard active now? - Boolean isActive = getShardActive(shardId); - - if (isActive == null) { - // shard was closed.. - continue; + long shardBytesUsed = getIndexBufferRAMBytesUsed(shardId); + if (shardBytesUsed > 0) { + totalBytesUsed += shardBytesUsed; } + } - // Was the shard active last time we checked? - Boolean wasActive = shardWasActive.get(shardId); + if (totalBytesUsed > indexingBuffer.bytes()) { + // OK we are using too much; make a queue and ask largest shard(s) to refresh: + logger.debug("now refreshing some shards: total indexing bytes used [{}] vs index_buffer_size [{}]", new ByteSizeValue(totalBytesUsed), indexingBuffer); - if (wasActive == null) { - // First time we are seeing this shard - shardWasActive.put(shardId, isActive); - changes.add(ShardStatusChangeType.ADDED); - } else if (isActive) { - // Shard is active now - if (wasActive == false) { - // Shard became active itself, since we last checked (due to new indexing op arriving) - changes.add(ShardStatusChangeType.BECAME_ACTIVE); - logger.debug("marking shard {} as active indexing wise", shardId); - shardWasActive.put(shardId, true); - } else if (checkIdle(shardId, inactiveTime.nanos()) == Boolean.TRUE) { - // Make shard inactive now - changes.add(ShardStatusChangeType.BECAME_INACTIVE); - logger.debug("marking shard {} as inactive (inactive_time[{}]) indexing wise", - shardId, - inactiveTime); - shardWasActive.put(shardId, false); + PriorityQueue queue = new PriorityQueue<>(); + for (ShardId shardId : availableShards()) { + long shardBytesUsed = getIndexBufferRAMBytesUsed(shardId); + if (shardBytesUsed > 0) { + queue.add(new ShardAndBytesUsed(shardBytesUsed, shardId)); } } - } - } - /** - * purge any existing statuses that are no longer updated - * - * @return the changes applied - */ - private EnumSet purgeDeletedAndClosedShards() { - EnumSet changes = EnumSet.noneOf(ShardStatusChangeType.class); - - Iterator statusShardIdIterator = shardWasActive.keySet().iterator(); - while (statusShardIdIterator.hasNext()) { - ShardId shardId = statusShardIdIterator.next(); - if (shardAvailable(shardId) == false) { - changes.add(ShardStatusChangeType.DELETED); - statusShardIdIterator.remove(); - } - } - return changes; - } - - private void calcAndSetShardBuffers(String reason) { - - // Count how many shards are now active: - int activeShardCount = 0; - for (Map.Entry ent : shardWasActive.entrySet()) { - if (ent.getValue()) { - activeShardCount++; - } - } - - // 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) { - logger.debug("no active shards (reason={})", reason); - return; - } - - ByteSizeValue shardIndexingBufferSize = new ByteSizeValue(indexingBuffer.bytes() / activeShardCount); - if (shardIndexingBufferSize.bytes() < minShardIndexBufferSize.bytes()) { - shardIndexingBufferSize = minShardIndexBufferSize; - } - if (shardIndexingBufferSize.bytes() > maxShardIndexBufferSize.bytes()) { - shardIndexingBufferSize = maxShardIndexBufferSize; - } - - ByteSizeValue shardTranslogBufferSize = new ByteSizeValue(translogBuffer.bytes() / activeShardCount); - if (shardTranslogBufferSize.bytes() < minShardTranslogBufferSize.bytes()) { - shardTranslogBufferSize = minShardTranslogBufferSize; - } - if (shardTranslogBufferSize.bytes() > maxShardTranslogBufferSize.bytes()) { - shardTranslogBufferSize = maxShardTranslogBufferSize; - } - - logger.debug("recalculating shard indexing buffer (reason={}), total is [{}] with [{}] active shards, each shard set to indexing=[{}], translog=[{}]", reason, indexingBuffer, activeShardCount, shardIndexingBufferSize, shardTranslogBufferSize); - - for (Map.Entry ent : shardWasActive.entrySet()) { - if (ent.getValue()) { - // This shard is active - updateShardBuffers(ent.getKey(), shardIndexingBufferSize, shardTranslogBufferSize); + while (totalBytesUsed > indexingBuffer.bytes() && queue.isEmpty() == false) { + ShardAndBytesUsed largest = queue.poll(); + logger.debug("refresh shard [{}] to free up its [{}] indexing buffer", largest.shardId, new ByteSizeValue(largest.bytesUsed)); + refreshShardAsync(largest.shardId); + totalBytesUsed -= largest.bytesUsed; } } } } - - protected long currentTimeInNanos() { - return System.nanoTime(); - } - - /** ask this shard to check now whether it is inactive, and reduces its indexing and translog buffers if so. returns Boolean.TRUE if - * it did deactive, Boolean.FALSE if it did not, and null if the shard is unknown */ - protected Boolean checkIdle(ShardId shardId, long inactiveTimeNS) { - String ignoreReason = null; - final IndexShard shard = getShard(shardId); - if (shard != null) { - try { - return shard.checkIdle(inactiveTimeNS); - } catch (EngineClosedException e) { - // ignore - ignoreReason = "EngineClosedException"; - } catch (FlushNotAllowedEngineException e) { - // ignore - ignoreReason = "FlushNotAllowedEngineException"; - } - } else { - ignoreReason = "shard not found"; - } - if (ignoreReason != null) { - logger.trace("ignore [{}] while marking shard {} as inactive", ignoreReason, shardId); - } - return null; - } - - private static enum ShardStatusChangeType { - ADDED, DELETED, BECAME_ACTIVE, BECAME_INACTIVE - } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineSettingsTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineSettingsTests.java index 1ed022dbefa..1d4eb6159e7 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineSettingsTests.java @@ -44,8 +44,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++) { @@ -55,15 +53,9 @@ 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_COMPOUND_ON_FLUSH, compoundOnFlush) .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()); @@ -77,12 +69,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() @@ -107,37 +93,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 - } } - - } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 5f6e1db42b9..a224bc87fec 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -444,7 +444,7 @@ public class InternalEngineTests extends ESTestCase { @Test 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); @@ -779,7 +779,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); @@ -985,7 +985,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++) { @@ -1435,7 +1435,7 @@ public class InternalEngineTests extends ESTestCase { @Test 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 @@ -1562,10 +1562,11 @@ public class InternalEngineTests extends ESTestCase { // #10312 @Test public void testDeletesAloneCanTriggerRefresh() throws Exception { + Settings settings = Settings.builder() + .put(defaultSettings) + .put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb").build(); 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)); + Engine engine = new InternalEngine(config(settings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()), false)) { 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); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b891219d636..ce3e4e51aba 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -339,8 +339,6 @@ 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); assertBusy(new Runnable() { // should be very very quick @Override public void run() { diff --git a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerIT.java b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerIT.java deleted file mode 100644 index e14cc226646..00000000000 --- a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerIT.java +++ /dev/null @@ -1,104 +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.memory; - -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.engine.EngineConfig; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.node.internal.InternalSettingsPreparer; -import org.elasticsearch.test.ESIntegTestCase; -import org.junit.Test; - - -@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(); - } - - @Test - public void testIndexBufferPushedToEngine() throws InterruptedException { - createNode(Settings.builder().put(IndexingMemoryController.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")); - } - - } - - @Test - public void testInactivePushedToShard() throws InterruptedException { - createNode(Settings.builder().put(IndexingMemoryController.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) - ); - } -} diff --git a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java index d3d9e961a61..033d625a26d 100644 --- a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java @@ -41,115 +41,80 @@ public class IndexingMemoryControllerTests extends ESTestCase { static class MockController extends IndexingMemoryController { - final static ByteSizeValue INACTIVE = new ByteSizeValue(-1); - - final Map indexingBuffers = new HashMap<>(); - final Map translogBuffers = new HashMap<>(); - - final Map lastIndexTimeNanos = new HashMap<>(); - final Set activeShards = new HashSet<>(); - - long currentTimeSec = TimeValue.timeValueNanos(System.nanoTime()).seconds(); + final Map indexBufferRAMBytesUsed = new HashMap<>(); public MockController(Settings settings) { super(Settings.builder() - .put(SHARD_INACTIVE_INTERVAL_TIME_SETTING, "200h") // disable it - .put(SHARD_INACTIVE_TIME_SETTING, "1ms") // nearly immediate + .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(ShardId id) { - indexingBuffers.remove(id); - translogBuffers.remove(id); - } - - public void assertBuffers(ShardId id, ByteSizeValue indexing, ByteSizeValue translog) { - assertThat(indexingBuffers.get(id), equalTo(indexing)); - assertThat(translogBuffers.get(id), equalTo(translog)); - } - - public void assertInActive(ShardId id) { - assertThat(indexingBuffers.get(id), equalTo(INACTIVE)); - assertThat(translogBuffers.get(id), equalTo(INACTIVE)); - } - - @Override - protected long currentTimeInNanos() { - return TimeValue.timeValueSeconds(currentTimeSec).nanos(); + public void deleteShard(ShardId shardId) { + indexBufferRAMBytesUsed.remove(shardId); } @Override protected List availableShards() { - return new ArrayList<>(indexingBuffers.keySet()); + return new ArrayList<>(indexBufferRAMBytesUsed.keySet()); } @Override protected boolean shardAvailable(ShardId shardId) { - return indexingBuffers.containsKey(shardId); + return indexBufferRAMBytesUsed.containsKey(shardId); } @Override - protected Boolean getShardActive(ShardId shardId) { - return activeShards.contains(shardId); - } - - @Override - protected void updateShardBuffers(ShardId shardId, ByteSizeValue shardIndexingBufferSize, ByteSizeValue shardTranslogBufferSize) { - indexingBuffers.put(shardId, shardIndexingBufferSize); - translogBuffers.put(shardId, shardTranslogBufferSize); - } - - @Override - protected Boolean checkIdle(ShardId shardId, long inactiveTimeNS) { - Long ns = lastIndexTimeNanos.get(shardId); - if (ns == null) { - return null; - } else if (currentTimeInNanos() - ns >= inactiveTimeNS) { - indexingBuffers.put(shardId, INACTIVE); - translogBuffers.put(shardId, INACTIVE); - activeShards.remove(shardId); - return true; + protected long getIndexBufferRAMBytesUsed(ShardId shardId) { + Long used = indexBufferRAMBytesUsed.get(shardId); + if (used == null) { + return 0; } else { - return false; + return used; } } - public void incrementTimeSec(int sec) { - currentTimeSec += sec; + @Override + public void refreshShardAsync(ShardId shardId) { + indexBufferRAMBytesUsed.put(shardId, 0L); + } + + public void assertBuffer(ShardId shardId, ByteSizeValue expected) { + Long actual = indexBufferRAMBytesUsed.get(shardId); + assertEquals(expected.bytes(), actual.longValue()); } public void simulateIndexing(ShardId shardId) { - lastIndexTimeNanos.put(shardId, currentTimeInNanos()); - if (indexingBuffers.containsKey(shardId) == false) { - // First time we are seeing this shard; start it off with inactive buffers as IndexShard does: - indexingBuffers.put(shardId, IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER); - translogBuffers.put(shardId, IndexingMemoryController.INACTIVE_SHARD_TRANSLOG_BUFFER); + Long bytes = indexBufferRAMBytesUsed.get(shardId); + if (bytes == null) { + bytes = 0L; } - activeShards.add(shardId); + // Each doc we index takes up a megabyte! + bytes += 1024*1024; + indexBufferRAMBytesUsed.put(shardId, bytes); forceCheck(); } } public void testShardAdditionAndRemoval() { MockController controller = new MockController(Settings.builder() - .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb") - .put(IndexingMemoryController.TRANSLOG_BUFFER_SIZE_SETTING, "100kb").build()); + .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "4mb").build()); final ShardId shard1 = new ShardId("test", 1); controller.simulateIndexing(shard1); - controller.assertBuffers(shard1, new ByteSizeValue(10, ByteSizeUnit.MB), new ByteSizeValue(64, ByteSizeUnit.KB)); // translog is maxed at 64K + controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); + // add another shard final ShardId shard2 = new ShardId("test", 2); controller.simulateIndexing(shard2); - controller.assertBuffers(shard1, new ByteSizeValue(5, ByteSizeUnit.MB), new ByteSizeValue(50, ByteSizeUnit.KB)); - controller.assertBuffers(shard2, new ByteSizeValue(5, ByteSizeUnit.MB), new ByteSizeValue(50, ByteSizeUnit.KB)); + controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard2, new ByteSizeValue(1, ByteSizeUnit.MB)); // remove first shard controller.deleteShard(shard1); controller.forceCheck(); - controller.assertBuffers(shard2, new ByteSizeValue(10, ByteSizeUnit.MB), new ByteSizeValue(64, ByteSizeUnit.KB)); // translog is maxed at 64K + controller.assertBuffer(shard2, new ByteSizeValue(1, ByteSizeUnit.MB)); // remove second shard controller.deleteShard(shard2); @@ -158,116 +123,62 @@ public class IndexingMemoryControllerTests extends ESTestCase { // add a new one final ShardId shard3 = new ShardId("test", 3); controller.simulateIndexing(shard3); - controller.assertBuffers(shard3, new ByteSizeValue(10, ByteSizeUnit.MB), new ByteSizeValue(64, ByteSizeUnit.KB)); // translog is maxed at 64K + controller.assertBuffer(shard3, new ByteSizeValue(1, ByteSizeUnit.MB)); } public void testActiveInactive() { MockController controller = new MockController(Settings.builder() - .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb") - .put(IndexingMemoryController.TRANSLOG_BUFFER_SIZE_SETTING, "100kb") - .put(IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, "5s") + .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "5mb") .build()); final ShardId shard1 = new ShardId("test", 1); controller.simulateIndexing(shard1); final ShardId shard2 = new ShardId("test", 2); controller.simulateIndexing(shard2); - controller.assertBuffers(shard1, new ByteSizeValue(5, ByteSizeUnit.MB), new ByteSizeValue(50, ByteSizeUnit.KB)); - controller.assertBuffers(shard2, new ByteSizeValue(5, ByteSizeUnit.MB), new ByteSizeValue(50, ByteSizeUnit.KB)); + controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard2, new ByteSizeValue(1, ByteSizeUnit.MB)); - // index into both shards, move the clock and see that they are still active controller.simulateIndexing(shard1); controller.simulateIndexing(shard2); - controller.incrementTimeSec(10); - controller.forceCheck(); + controller.assertBuffer(shard1, new ByteSizeValue(2, ByteSizeUnit.MB)); + controller.assertBuffer(shard2, new ByteSizeValue(2, ByteSizeUnit.MB)); - // both shards now inactive - controller.assertInActive(shard1); - controller.assertInActive(shard2); - - // index into one shard only, see it becomes active + // index into one shard only, hits the 5mb limit, so shard1 is refreshed controller.simulateIndexing(shard1); - controller.assertBuffers(shard1, new ByteSizeValue(10, ByteSizeUnit.MB), new ByteSizeValue(64, ByteSizeUnit.KB)); - controller.assertInActive(shard2); + controller.assertBuffer(shard1, new ByteSizeValue(0, ByteSizeUnit.MB)); + controller.assertBuffer(shard2, new ByteSizeValue(2, ByteSizeUnit.MB)); - controller.incrementTimeSec(3); // increment but not enough to become inactive - controller.forceCheck(); - controller.assertBuffers(shard1, new ByteSizeValue(10, ByteSizeUnit.MB), new ByteSizeValue(64, ByteSizeUnit.KB)); - controller.assertInActive(shard2); - - controller.incrementTimeSec(3); // increment some more - controller.forceCheck(); - controller.assertInActive(shard1); - controller.assertInActive(shard2); - - // index some and shard becomes immediately active controller.simulateIndexing(shard2); - controller.assertInActive(shard1); - controller.assertBuffers(shard2, new ByteSizeValue(10, ByteSizeUnit.MB), new ByteSizeValue(64, ByteSizeUnit.KB)); - } - - public void testMinShardBufferSizes() { - MockController controller = new MockController(Settings.builder() - .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "10mb") - .put(IndexingMemoryController.TRANSLOG_BUFFER_SIZE_SETTING, "50kb") - .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.TRANSLOG_BUFFER_SIZE_SETTING, "50kb") - .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)); + controller.simulateIndexing(shard2); + controller.assertBuffer(shard2, new ByteSizeValue(4, ByteSizeUnit.MB)); + controller.simulateIndexing(shard2); + // shard2 used up the full 5 mb and is now cleared: + controller.assertBuffer(shard2, new ByteSizeValue(0, ByteSizeUnit.MB)); } public void testRelativeBufferSizes() { MockController controller = new MockController(Settings.builder() .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "50%") - .put(IndexingMemoryController.TRANSLOG_BUFFER_SIZE_SETTING, "0.5%") .build()); assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(50, ByteSizeUnit.MB))); - assertThat(controller.translogBufferSize(), equalTo(new ByteSizeValue(512, ByteSizeUnit.KB))); } public void testMinBufferSizes() { MockController controller = new MockController(Settings.builder() .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "0.001%") - .put(IndexingMemoryController.TRANSLOG_BUFFER_SIZE_SETTING, "0.001%") - .put(IndexingMemoryController.MIN_INDEX_BUFFER_SIZE_SETTING, "6mb") - .put(IndexingMemoryController.MIN_TRANSLOG_BUFFER_SIZE_SETTING, "512kb").build()); + .put(IndexingMemoryController.MIN_INDEX_BUFFER_SIZE_SETTING, "6mb").build()); assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(6, ByteSizeUnit.MB))); - assertThat(controller.translogBufferSize(), equalTo(new ByteSizeValue(512, ByteSizeUnit.KB))); } public void testMaxBufferSizes() { MockController controller = new MockController(Settings.builder() .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "90%") - .put(IndexingMemoryController.TRANSLOG_BUFFER_SIZE_SETTING, "90%") - .put(IndexingMemoryController.MAX_INDEX_BUFFER_SIZE_SETTING, "6mb") - .put(IndexingMemoryController.MAX_TRANSLOG_BUFFER_SIZE_SETTING, "512kb").build()); + .put(IndexingMemoryController.MAX_INDEX_BUFFER_SIZE_SETTING, "6mb").build()); assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(6, ByteSizeUnit.MB))); - assertThat(controller.translogBufferSize(), equalTo(new ByteSizeValue(512, ByteSizeUnit.KB))); } - - protected void assertTwoActiveShards(MockController controller, ByteSizeValue indexBufferSize, ByteSizeValue translogBufferSize) { - final ShardId shard1 = new ShardId("test", 1); - controller.simulateIndexing(shard1); - final ShardId shard2 = new ShardId("test", 2); - controller.simulateIndexing(shard2); - controller.assertBuffers(shard1, indexBufferSize, translogBufferSize); - controller.assertBuffers(shard2, indexBufferSize, translogBufferSize); - - } - } From 6ae8ca9a5e8e56d012407237453303f1c2ff82f2 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 14 Oct 2015 06:18:25 -0400 Subject: [PATCH 02/33] put back active/inactive logic, for sync'd flush --- .../elasticsearch/index/shard/IndexShard.java | 41 +++++++++++++++++++ .../memory/IndexingMemoryController.java | 28 ++++++++++--- .../index/engine/InternalEngineTests.java | 4 +- .../index/shard/IndexShardTests.java | 2 + .../memory/IndexingMemoryControllerTests.java | 11 +++-- 5 files changed, 77 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 292ef01b8d1..fee033feef3 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -189,6 +189,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett private final IndexSearcherWrapper searcherWrapper; + /** True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link + * IndexingMemoryController}). */ + private final AtomicBoolean active = new AtomicBoolean(); + + private volatile long lastWriteNS; private final IndexingMemoryController indexingMemoryController; @Inject @@ -450,6 +455,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett */ public boolean index(Engine.Index index) { ensureWriteAllowed(index); + markLastWrite(index); index = indexingService.preIndex(index); final boolean created; try { @@ -474,6 +480,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett public void delete(Engine.Delete delete) { ensureWriteAllowed(delete); + markLastWrite(delete); delete = indexingService.preDelete(delete); try { if (logger.isTraceEnabled()) { @@ -883,6 +890,22 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett } } + /** Returns timestamp of last indexing operation */ + public long getLastWriteNS() { + return lastWriteNS; + } + + /** Records timestamp of the last write operation, possibly switching {@code active} to true if we were inactive. */ + private void markLastWrite(Engine.Operation op) { + lastWriteNS = op.startTime(); + if (active.getAndSet(true) == false) { + // We are currently inactive, but a new write operation just showed up, so we now notify IMC + // to wake up and fix our indexing buffer. We could do this async instead, but cost should + // be low, and it's rare this happens. + indexingMemoryController.forceCheck(); + } + } + private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardStateException { Engine.Operation.Origin origin = op.origin(); IndexShardState state = this.state; // one time volatile read @@ -954,6 +977,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett return engine.indexBufferRAMBytesUsed(); } + /** 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. */ + public void checkIdle(long inactiveTimeNS) { + if (System.nanoTime() - lastWriteNS >= inactiveTimeNS) { + boolean wasActive = active.getAndSet(false); + if (wasActive) { + logger.debug("shard is now inactive"); + indicesLifecycle.onShardInactive(this); + } + } + } + + /** Returns {@code true} if this shard is active (has seen indexing ops in the last {@link + * IndexingMemoryController#SHARD_INACTIVE_TIME_SETTING} (default 5 minutes), else {@code false}. */ + public boolean getActive() { + return active.get(); + } + public final boolean isFlushOnClose() { return flushOnClose; } diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index aebaeccf483..821311ad636 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -51,6 +51,9 @@ public class IndexingMemoryController extends AbstractLifecycleComponentindices.memory.index_buffer_size 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"; + /** 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"; + /** How frequently we check indexing memory usage (default: 5 seconds). */ public static final String SHARD_MEMORY_INTERVAL_TIME_SETTING = "indices.memory.interval"; @@ -62,6 +65,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent 0) { - totalBytesUsed += shardBytesUsed; - } + + // Give shard a chance to transition to inactive so sync'd flush can happen: + checkIdle(shardId, inactiveTime.nanos()); + + totalBytesUsed += getIndexBufferRAMBytesUsed(shardId); } if (totalBytesUsed > indexingBuffer.bytes()) { diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index a224bc87fec..2b929f76ad3 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -91,6 +91,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogTests; +import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -1564,7 +1565,8 @@ public class InternalEngineTests extends ESTestCase { public void testDeletesAloneCanTriggerRefresh() throws Exception { Settings settings = Settings.builder() .put(defaultSettings) - .put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb").build(); + .put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb") + .put(IndexingMemoryController.SHARD_MEMORY_INTERVAL_TIME_SETTING, "100ms").build(); try (Store store = createStore(); Engine engine = new InternalEngine(config(settings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()), false)) { for (int i = 0; i < 100; i++) { diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index ce3e4e51aba..fd402130092 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -339,6 +339,8 @@ public class IndexShardTests extends ESSingleNodeTestCase { client().prepareIndex("test", "test").setSource("{}").get(); ensureGreen("test"); IndicesService indicesService = getInstanceFromNode(IndicesService.class); + // force the shard to become idle now: + indicesService.indexService("test").getShardOrNull(0).checkIdle(0); assertBusy(new Runnable() { // should be very very quick @Override public void run() { diff --git a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java index 033d625a26d..2d54fb0a1fd 100644 --- a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java @@ -75,6 +75,10 @@ public class IndexingMemoryControllerTests extends ESTestCase { } } + @Override + protected void checkIdle(ShardId shardId, long inactiveTimeNS) { + } + @Override public void refreshShardAsync(ShardId shardId) { indexBufferRAMBytesUsed.put(shardId, 0L); @@ -103,7 +107,6 @@ public class IndexingMemoryControllerTests extends ESTestCase { final ShardId shard1 = new ShardId("test", 1); controller.simulateIndexing(shard1); controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); - // add another shard final ShardId shard2 = new ShardId("test", 2); @@ -144,7 +147,8 @@ public class IndexingMemoryControllerTests extends ESTestCase { controller.assertBuffer(shard1, new ByteSizeValue(2, ByteSizeUnit.MB)); controller.assertBuffer(shard2, new ByteSizeValue(2, ByteSizeUnit.MB)); - // index into one shard only, hits the 5mb limit, so shard1 is refreshed + // index into one shard only, crosses the 5mb limit, so shard1 is refreshed + controller.simulateIndexing(shard1); controller.simulateIndexing(shard1); controller.assertBuffer(shard1, new ByteSizeValue(0, ByteSizeUnit.MB)); controller.assertBuffer(shard2, new ByteSizeValue(2, ByteSizeUnit.MB)); @@ -153,7 +157,8 @@ public class IndexingMemoryControllerTests extends ESTestCase { controller.simulateIndexing(shard2); controller.assertBuffer(shard2, new ByteSizeValue(4, ByteSizeUnit.MB)); controller.simulateIndexing(shard2); - // shard2 used up the full 5 mb and is now cleared: + controller.simulateIndexing(shard2); + // shard2 crossed 5 mb and is now cleared: controller.assertBuffer(shard2, new ByteSizeValue(0, ByteSizeUnit.MB)); } From 1b9e9ed0920dcd6270cf570041d53896b9a7c139 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 14 Oct 2015 09:11:31 -0400 Subject: [PATCH 03/33] fix failing test --- .../metadata/MetaDataIndexUpgradeService.java | 3 +- .../index/engine/EngineConfig.java | 7 ++- .../memory/IndexingMemoryController.java | 4 +- .../index/engine/InternalEngineTests.java | 45 ++++++++++++------- 4 files changed, 37 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index 29a3dcaeb9f..cdde49170d4 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -125,8 +125,7 @@ public class MetaDataIndexUpgradeService extends AbstractComponent { "index.store.throttle.max_bytes_per_sec", "index.translog.flush_threshold_size", "index.translog.fs.buffer_size", - "index.version_map_size", - "index.buffer_size")); + "index.version_map_size")); /** All known time settings for an index. */ public static final Set INDEX_TIME_SETTINGS = unmodifiableSet(newHashSet( diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index e76aa1d38b9..abca9cc875e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -97,7 +97,6 @@ public final class EngineConfig { * Index setting to control the index buffer size. * This setting is not realtime updateable. */ - public static final String INDEX_BUFFER_SIZE_SETTING = "index.buffer_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"; @@ -132,9 +131,9 @@ public final class EngineConfig { this.failedEngineListener = failedEngineListener; this.compoundOnFlush = indexSettings.getAsBoolean(INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); codecName = indexSettings.get(INDEX_CODEC_SETTING, DEFAULT_CODEC_NAME); - // We tell IndexWriter to use large heap, but IndexingMemoryController checks periodically and refreshes the most heap-consuming - // shards when total indexing heap usage is too high: - indexingBufferSize = indexSettings.getAsBytesSize(INDEX_BUFFER_SIZE_SETTING, new ByteSizeValue(256, ByteSizeUnit.MB)); + // We give IndexWriter a huge buffer, so it won't flush on its own. Instead, IndexingMemoryController periodically checks + // and refreshes the most heap-consuming shards when total indexing heap usage is too high: + indexingBufferSize = new ByteSizeValue(256, ByteSizeUnit.MB); gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, DEFAULT_GC_DELETES).millis(); this.translogRecoveryPerformer = translogRecoveryPerformer; this.forceNewTranslog = indexSettings.getAsBoolean(INDEX_FORCE_NEW_TRANSLOG, false); diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 821311ad636..6073aa54626 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -170,7 +170,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()) { // OK we are using too much; make a queue and ask largest shard(s) to refresh: logger.debug("now refreshing some shards: total indexing bytes used [{}] vs index_buffer_size [{}]", new ByteSizeValue(totalBytesUsed), indexingBuffer); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 2b929f76ad3..9715498a90c 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1565,10 +1565,9 @@ public class InternalEngineTests extends ESTestCase { public void testDeletesAloneCanTriggerRefresh() throws Exception { Settings settings = Settings.builder() .put(defaultSettings) - .put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb") - .put(IndexingMemoryController.SHARD_MEMORY_INTERVAL_TIME_SETTING, "100ms").build(); + .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "1kb").build(); try (Store store = createStore(); - Engine engine = new InternalEngine(config(settings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()), false)) { + Engine engine = new InternalEngine(config(settings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()), false)) { 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); @@ -1578,6 +1577,30 @@ public class InternalEngineTests extends ESTestCase { // Force merge so we know all merges are done before we start deleting: engine.forceMerge(true, 1, false, false, false); + // Make a shell of an IMC to check up on indexing buffer usage: + IndexingMemoryController imc = new IndexingMemoryController(settings, threadPool, null) { + @Override + protected IndexShard getShard(ShardId shardId) { + return null; + } + + @Override + protected List availableShards() { + return Collections.singletonList(new ShardId("foo", 0)); + } + + @Override + protected void refreshShardAsync(ShardId shardId) { + engine.refresh("memory"); + } + + @Override + protected long getIndexBufferRAMBytesUsed(ShardId shardId) { + System.out.println("BYTES USED: " + engine.indexBufferRAMBytesUsed()); + return engine.indexBufferRAMBytesUsed(); + } + }; + Searcher s = engine.acquireSearcher("test"); final long version1 = ((DirectoryReader) s.reader()).getVersion(); s.close(); @@ -1586,18 +1609,10 @@ public class InternalEngineTests extends ESTestCase { 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)); - } - }); + imc.forceCheck(); + try (Searcher s2 = engine.acquireSearcher("test")) { + assertThat(((DirectoryReader) s2.reader()).getVersion(), greaterThan(version1)); + } } } From 77c2445f7d733b2e281abe9f3b9dc1868648a19e Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 14 Oct 2015 16:44:22 -0400 Subject: [PATCH 04/33] don't call IMC.forceCheck when going active; remove nocommit/sops --- .../ElasticsearchConcurrentMergeScheduler.java | 2 +- .../org/elasticsearch/index/shard/IndexShard.java | 13 +++---------- .../indices/memory/IndexingMemoryController.java | 7 ++----- 3 files changed, 6 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java b/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java index 81466128c56..0566e80a37d 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java @@ -178,7 +178,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(); } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index fee033feef3..d4d4330a68b 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -179,7 +179,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett public static final String INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS = "index.translog.flush_threshold_ops"; public static final String INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE = "index.translog.flush_threshold_size"; public static final String INDEX_TRANSLOG_DISABLE_FLUSH = "index.translog.disable_flush"; - + public static final String INDEX_REFRESH_INTERVAL = "index.refresh_interval"; private final ShardPath path; @@ -898,12 +898,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett /** Records timestamp of the last write operation, possibly switching {@code active} to true if we were inactive. */ private void markLastWrite(Engine.Operation op) { lastWriteNS = op.startTime(); - if (active.getAndSet(true) == false) { - // We are currently inactive, but a new write operation just showed up, so we now notify IMC - // to wake up and fix our indexing buffer. We could do this async instead, but cost should - // be low, and it's rare this happens. - indexingMemoryController.forceCheck(); - } } private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardStateException { @@ -963,8 +957,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett } } - public static final String INDEX_REFRESH_INTERVAL = "index.refresh_interval"; - public void addFailedEngineListener(Engine.FailedEngineListener failedEngineListener) { this.failedEngineListener.delegates.add(failedEngineListener); } @@ -1177,7 +1169,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett * changes. */ public void refreshAsync(final String reason) { - // nocommit this really is async??? engineConfig.getThreadPool().executor(ThreadPool.Names.REFRESH).execute(new Runnable() { @Override public void run() { @@ -1202,6 +1193,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett @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"); } diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 6073aa54626..5519fdd9fd7 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -54,7 +54,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()) { // OK we are using too much; make a queue and ask largest shard(s) to refresh: logger.debug("now refreshing some shards: total indexing bytes used [{}] vs index_buffer_size [{}]", new ByteSizeValue(totalBytesUsed), indexingBuffer); - PriorityQueue queue = new PriorityQueue<>(); for (ShardId shardId : availableShards()) { long shardBytesUsed = getIndexBufferRAMBytesUsed(shardId); From 0a2a7f2f7a2b19b4cbd5b954f028d9b5a9201fd9 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 14 Oct 2015 17:04:13 -0400 Subject: [PATCH 05/33] set active to true on indexing ops --- core/src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 0c22a8e3728..5965ea4e245 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -909,6 +909,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett /** Records timestamp of the last write operation, possibly switching {@code active} to true if we were inactive. */ private void markLastWrite(Engine.Operation op) { lastWriteNS = op.startTime(); + active.set(true); } private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardStateException { From c66b05d9cf7228a097e68484caa531695f2d49ae Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Thu, 5 Nov 2015 16:04:48 -0500 Subject: [PATCH 06/33] force check after X bytes indexed --- .../elasticsearch/index/engine/Engine.java | 25 ++++++------- .../index/engine/InternalEngine.java | 4 +++ .../elasticsearch/index/shard/IndexShard.java | 10 ++++-- .../memory/IndexingMemoryController.java | 36 +++++++++++++++++-- 4 files changed, 58 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 56db9cb3e5b..62e7ec81f46 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -19,11 +19,23 @@ package org.elasticsearch.index.engine; +import java.io.Closeable; +import java.io.IOException; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; + import org.apache.lucene.index.*; 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; @@ -49,17 +61,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; -import java.io.Closeable; -import java.io.IOException; -import java.util.*; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Function; - /** * */ @@ -361,7 +362,7 @@ public abstract class Engine implements Closeable { stats.addIndexWriterMaxMemoryInBytes(0); } - /** How much heap is used that would be freed by a refresh */ + /** How much heap is used that would be freed by a refresh. Note that this may throw {@link AlreadyClosedException}. */ abstract public long indexBufferRAMBytesUsed(); protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 128195b6deb..b91e60d1d2a 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -760,6 +760,10 @@ public class InternalEngine extends Engine { @Override public long indexBufferRAMBytesUsed() { if (refreshing) { + // We tell a "white lie" here, pretending that we instantaneously moved all + // heap to disk at the start of refresh. We do this so IMC behaves as if we + // are using no heap, else it will just keep asking us when it should be + // asking others: return 0; } else { return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 5965ea4e245..b07ad5fc566 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -471,6 +471,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett throw ex; } indexingService.postIndex(index); + indexingMemoryController.bytesWritten(index.getTranslogLocation().size); return created; } @@ -495,6 +496,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett throw ex; } indexingService.postDelete(delete); + indexingMemoryController.bytesWritten(delete.getTranslogLocation().size); } public Engine.GetResult get(Engine.Get get) { @@ -978,7 +980,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett if (engine == null) { return 0; } - return engine.indexBufferRAMBytesUsed(); + try { + return engine.indexBufferRAMBytesUsed(); + } catch (AlreadyClosedException ex) { + return 0; + } } /** Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last @@ -1193,7 +1199,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett }); } - 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 diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 5519fdd9fd7..d9874e6d47e 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -54,7 +54,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent { + long startMS = System.currentTimeMillis(); + + /** called by IndexShard to record that this many bytes were written to translog */ + public void bytesWritten(int bytes) { + statusChecker.bytesWritten(bytes); + } + + static final class ShardAndBytesUsed implements Comparable { final long bytesUsed; final ShardId shardId; @@ -225,9 +232,26 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()/20) { + // NOTE: this is only an approximate check, because bytes written is to the translog, vs indexing memory buffer which is + // typically smaller. But this logic is here only as a safety against thread starvation or too infrequent checking, + // to ensure we are still checking in proportion to bytes processed by indexing: + System.out.println(((System.currentTimeMillis() - startMS)/1000.0) + ": NOW CHECK xlog=" + bytesWrittenSinceCheck); + run(); + } + } + @Override public synchronized void run() { + // nocommit lower the translog buffer to 8 KB + + // nocommit add defensive try/catch-everything here? bad if an errant EngineClosedExc kills off this thread!! + // Fast check to sum up how much heap all shards' indexing buffers are using now: long totalBytesUsed = 0; for (ShardId shardId : availableShards()) { @@ -236,8 +260,11 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()) { // OK we are using too much; make a queue and ask largest shard(s) to refresh: logger.debug("now refreshing some shards: total indexing bytes used [{}] vs index_buffer_size [{}]", new ByteSizeValue(totalBytesUsed), indexingBuffer); @@ -251,11 +278,14 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes() && queue.isEmpty() == false) { ShardAndBytesUsed largest = queue.poll(); + System.out.println("IMC: write " + largest.shardId + ": " + (largest.bytesUsed/1024./1024.) + " MB"); logger.debug("refresh shard [{}] to free up its [{}] indexing buffer", largest.shardId, new ByteSizeValue(largest.bytesUsed)); refreshShardAsync(largest.shardId); totalBytesUsed -= largest.bytesUsed; } } + + bytesWrittenSinceCheck = 0; } } } From 99e328c9bfb87a84943ab3357d5d2f0cc07c12a5 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Mon, 14 Dec 2015 15:39:18 -0500 Subject: [PATCH 07/33] remove 'white lie' and tracking refreshing bytes explicitly --- .../index/engine/InternalEngine.java | 15 +---- .../elasticsearch/index/shard/IndexShard.java | 17 ++++-- .../memory/IndexingMemoryController.java | 56 +++++++++++++++++-- 3 files changed, 63 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index b91e60d1d2a..089ddfc9998 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -102,8 +102,6 @@ public class InternalEngine extends Engine { private volatile SegmentInfos lastCommittedSegmentInfos; - private volatile boolean refreshing; - private final IndexThrottle throttle; public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogRecovery) throws EngineException { @@ -489,7 +487,6 @@ public class InternalEngine extends Engine { // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); - refreshing = true; searcherManager.maybeRefreshBlocking(); } catch (AlreadyClosedException e) { ensureOpen(); @@ -499,8 +496,6 @@ public class InternalEngine extends Engine { } catch (Throwable t) { failEngine("refresh failed", t); throw new RefreshFailedEngineException(shardId, t); - } finally { - refreshing = false; } // TODO: maybe we should just put a scheduled job in threadPool? @@ -759,15 +754,7 @@ public class InternalEngine extends Engine { @Override public long indexBufferRAMBytesUsed() { - if (refreshing) { - // We tell a "white lie" here, pretending that we instantaneously moved all - // heap to disk at the start of refresh. We do this so IMC behaves as if we - // are using no heap, else it will just keep asking us when it should be - // asking others: - return 0; - } else { - return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); - } + return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index b07ad5fc566..7caa98b6104 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -506,12 +506,19 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett public void refresh(String source) { verifyNotClosed(); - if (logger.isTraceEnabled()) { - logger.trace("refresh with source: {}", source); + // nocommit OK to throw EngineClosedExc? + long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); + indexingMemoryController.addRefreshingBytes(shardId, ramBytesUsed); + try { + if (logger.isTraceEnabled()) { + logger.trace("refresh with source: {} indexBufferRAMBytesUsed={}", source, ramBytesUsed); + } + long time = System.nanoTime(); + getEngine().refresh(source); + refreshMetric.inc(System.nanoTime() - time); + } finally { + indexingMemoryController.removeRefreshingBytes(shardId, ramBytesUsed); } - long time = System.nanoTime(); - getEngine().refresh(source); - refreshMetric.inc(System.nanoTime() - time); } public RefreshStats refreshStats() { diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index d9874e6d47e..b3c55ea92ea 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -58,7 +58,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent refreshingBytes = new ConcurrentHashMap<>(); + @Inject public IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) { this(settings, threadPool, indicesService, JvmInfo.jvmInfo().getMem().getHeapMax().bytes()); @@ -117,6 +122,15 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()) { + if (totalBytesUsed - bytesRefreshingNow.get() > indexingBuffer.bytes()) { // OK we are using too much; make a queue and ask largest shard(s) to refresh: logger.debug("now refreshing some shards: total indexing bytes used [{}] vs index_buffer_size [{}]", new ByteSizeValue(totalBytesUsed), indexingBuffer); PriorityQueue queue = new PriorityQueue<>(); for (ShardId shardId : availableShards()) { + // nocommit explain why order is important here! + Long refreshingBytes = refreshingBytes.get(shardId); + long shardBytesUsed = getIndexBufferRAMBytesUsed(shardId); + + if (refreshingBytes != null) { + // Only count up bytes not already being refreshed: + shardBytesUsed -= refreshingBytes; + + // If the refresh completed just after we pulled refreshingBytes and before we pulled index buffer bytes, then we could + // have a negative value here: + if (shardBytesUsed < 0) { + continue; + } + } + if (shardBytesUsed > 0) { queue.add(new ShardAndBytesUsed(shardBytesUsed, shardId)); } From 86a0dd0f28dc02524adf8c2a8156c59bf2937dbf Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Tue, 15 Dec 2015 06:29:45 -0500 Subject: [PATCH 08/33] use IW.flush to move bytes to disk --- .../elasticsearch/index/engine/Engine.java | 6 +++ .../index/engine/InternalEngine.java | 39 +++++++++++++++++++ .../index/engine/ShadowEngine.java | 6 +++ .../elasticsearch/index/shard/IndexShard.java | 39 +++++++++++++++---- .../memory/IndexingMemoryController.java | 19 +++++---- .../memory/IndexingMemoryControllerTests.java | 2 +- 6 files changed, 96 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index d6dd31481db..079bd277750 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -478,6 +478,12 @@ public abstract class Engine implements Closeable { */ 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. * diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 2aed653d02e..7b46dd3cc9a 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -506,6 +506,45 @@ public class InternalEngine extends Engine { mergeScheduler.refreshConfig(); } + @Override + public void writeIndexingBuffer() throws EngineException { + + // 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 asks for, and another for version + // map interactions: + boolean useRefresh = versionMapRefreshPending.get() || (indexWriter.ramBytesUsed()/4 < versionMap.ramBytesUsedForRefresh()); + + // we obtain a read lock here, since we don't want a flush to happen while we are refreshing + // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) + try (ReleasableLock lock = readLock.acquire()) { + ensureOpen(); + if (useRefresh) { + // The version map is using > 25% of the indexing buffer, so we do a refresh so the version map also clears + searcherManager.maybeRefreshBlocking(); + } 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: + 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); + } + + // TODO: maybe we should just put a scheduled job in threadPool? + // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes + // for a long time: + if (useRefresh) { + maybePruneDeletedTombstones(); + versionMapRefreshPending.set(false); + mergeScheduler.refreshConfig(); + } + } + @Override public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException { // best effort attempt before we acquire locks diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index dad1c5e09f2..8a013a31792 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -231,4 +231,10 @@ public class ShadowEngine extends Engine { // 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"); + } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index fcf204603f5..cf0296ef461 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -537,7 +537,7 @@ public class IndexShard extends AbstractIndexShardComponent { verifyNotClosed(); // nocommit OK to throw EngineClosedExc? long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); - indexingMemoryController.addRefreshingBytes(this, ramBytesUsed); + indexingMemoryController.addWritingBytes(this, ramBytesUsed); try { if (logger.isTraceEnabled()) { logger.trace("refresh with source: {} indexBufferRAMBytesUsed={}", source, ramBytesUsed); @@ -546,7 +546,7 @@ public class IndexShard extends AbstractIndexShardComponent { getEngine().refresh(source); refreshMetric.inc(System.nanoTime() - time); } finally { - indexingMemoryController.removeRefreshingBytes(this, ramBytesUsed); + indexingMemoryController.removeWritingBytes(this, ramBytesUsed); } } @@ -1210,17 +1210,42 @@ public class IndexShard extends AbstractIndexShardComponent { } /** - * Asynchronously refreshes the engine for new search operations to reflect the latest - * changes. + * Called when our shard is using too much heap and should move buffered indexed/deleted documents to disk. */ - public void refreshAsync(final String reason) { - engineConfig.getThreadPool().executor(ThreadPool.Names.REFRESH).execute(new Runnable() { + public void writeIndexingBufferAsync() { + threadPool.executor(ThreadPool.Names.REFRESH).execute(new Runnable() { @Override public void run() { try { - refresh(reason); + Engine engine = getEngine(); + long bytes = engine.indexBufferRAMBytesUsed(); + // NOTE: this can be an overestimate by up to 20%, if engine uses IW.flush not refresh, 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: + indexingMemoryController.addWritingBytes(IndexShard.this, bytes); + try { + getEngine().writeIndexingBuffer(); + } finally { + indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); + } } catch (EngineClosedException ex) { // 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); + } } } }); diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 53c549efc77..a699dabcfff 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -124,11 +124,14 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()/20) { // NOTE: this is only an approximate check, because bytes written is to the translog, vs indexing memory buffer which is - // typically smaller. But this logic is here only as a safety against thread starvation or too infrequent checking, - // to ensure we are still checking in proportion to bytes processed by indexing: + // 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: System.out.println(((System.currentTimeMillis() - startMS)/1000.0) + ": NOW CHECK xlog=" + bytesWrittenSinceCheck); run(); } @@ -293,7 +298,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent Date: Wed, 16 Dec 2015 06:41:24 -0500 Subject: [PATCH 09/33] trigger index throttling if writing indexing buffers to disk can't keep up and we are >= 50% over the allowed budget --- .../elasticsearch/index/engine/Engine.java | 10 ++ .../index/engine/InternalEngine.java | 32 ++++-- .../index/engine/ShadowEngine.java | 10 ++ .../elasticsearch/index/shard/IndexShard.java | 20 +++- .../memory/IndexingMemoryController.java | 102 ++++++++++++------ 5 files changed, 131 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 079bd277750..410d2af8d0b 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1087,4 +1087,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(); } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 7b46dd3cc9a..486920dc750 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -42,6 +42,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; @@ -510,9 +511,12 @@ public class InternalEngine extends Engine { public void writeIndexingBuffer() throws EngineException { // 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 asks for, and another for version - // map interactions: - boolean useRefresh = versionMapRefreshPending.get() || (indexWriter.ramBytesUsed()/4 < versionMap.ramBytesUsedForRefresh()); + // 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: + long versionMapBytes = versionMap.ramBytesUsedForRefresh(); + long indexingBufferBytes = indexWriter.ramBytesUsed(); + + boolean useRefresh = versionMapRefreshPending.get() || (indexingBufferBytes/4 < versionMapBytes); // we obtain a read lock here, since we don't want a flush to happen while we are refreshing // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) @@ -520,9 +524,13 @@ public class InternalEngine extends Engine { ensureOpen(); if (useRefresh) { // The version map is using > 25% of the indexing buffer, so we do a refresh so the version map also clears - searcherManager.maybeRefreshBlocking(); + 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 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) { @@ -1043,12 +1051,24 @@ public class InternalEngine extends Engine { } } + private final AtomicInteger throttleRequestCount = new AtomicInteger(); + + @Override public void activateThrottling() { - throttle.activate(); + int count = throttleRequestCount.incrementAndGet(); + assert count >= 1; + if (count == 1) { + throttle.activate(); + } } + @Override public void deactivateThrottling() { - throttle.deactivate(); + int count = throttleRequestCount.decrementAndGet(); + assert count >= 0; + if (count == 0) { + throttle.deactivate(); + } } long getGcDeletesInMillis() { diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 8a013a31792..286b06e69f9 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -237,4 +237,14 @@ public class ShadowEngine extends Engine { // 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"); + } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index cf0296ef461..3ac6d1ceefa 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -539,9 +539,7 @@ public class IndexShard extends AbstractIndexShardComponent { long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); indexingMemoryController.addWritingBytes(this, ramBytesUsed); try { - if (logger.isTraceEnabled()) { - logger.trace("refresh with source: {} indexBufferRAMBytesUsed={}", source, ramBytesUsed); - } + logger.debug("refresh with source: {} indexBufferRAMBytesUsed={}", source, ramBytesUsed); long time = System.nanoTime(); getEngine().refresh(source); refreshMetric.inc(System.nanoTime() - time); @@ -1209,6 +1207,22 @@ public class IndexShard extends AbstractIndexShardComponent { return indexEventListener; } + public void activateThrottling() { + try { + getEngine().activateThrottling(); + } catch (EngineClosedException ex) { + // ignore + } + } + + public void deactivateThrottling() { + try { + getEngine().deactivateThrottling(); + } catch (EngineClosedException ex) { + // ignore + } + } + /** * Called when our shard is using too much heap and should move buffered indexed/deleted documents to disk. */ diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index a699dabcfff..c3199426b13 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -70,6 +70,9 @@ public class IndexingMemoryController extends AbstractLifecycleComponent throttled = new HashSet<>(); + private volatile ScheduledFuture scheduler; private static final EnumSet CAN_UPDATE_INDEX_BUFFER_STATES = EnumSet.of( @@ -77,10 +80,8 @@ public class IndexingMemoryController extends AbstractLifecycleComponent refreshingBytes = new ConcurrentHashMap<>(); + /** Maps each shard to how many bytes it is currently, asynchronously, writing to disk */ + private final Map writingBytes = new ConcurrentHashMap<>(); @Inject public IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) { @@ -124,16 +125,18 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()/20) { + if (bytesWrittenSinceCheck > indexingBuffer.bytes()/30) { // 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: - System.out.println(((System.currentTimeMillis() - startMS)/1000.0) + ": NOW CHECK xlog=" + bytesWrittenSinceCheck); run(); } } @@ -237,69 +239,101 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()) { - // OK we are using too much; make a queue and ask largest shard(s) to refresh: - logger.debug("now refreshing some shards: total indexing bytes used [{}] vs index_buffer_size [{}]", new ByteSizeValue(totalBytesUsed), indexingBuffer); + if (totalBytesUsed > indexingBuffer.bytes()) { + // OK we are now over-budget; fill the priority queue and ask largest shard(s) to refresh: + logger.debug("now write some indexing buffers: total indexing heap bytes used [{}] vs {} [{}], currently writing bytes [{}]", + new ByteSizeValue(totalBytesUsed), INDEX_BUFFER_SIZE_SETTING, indexingBuffer, new ByteSizeValue(totalBytesWriting)); PriorityQueue queue = new PriorityQueue<>(); - for (IndexShard shard : availableShards()) { - // nocommit explain why order is important here! - Long bytes = refreshingBytes.get(shard); + for (IndexShard shard : availableShards()) { + // How many bytes this shard is currently (async'd) moving from heap to disk: + Long shardWritingBytes = writingBytes.get(shard); + + // How many heap bytes this shard is currently using long shardBytesUsed = getIndexBufferRAMBytesUsed(shard); - if (bytes != null) { + if (shardWritingBytes != null) { // Only count up bytes not already being refreshed: - shardBytesUsed -= bytes; + shardBytesUsed -= shardWritingBytes; - // If the refresh completed just after we pulled refreshingBytes and before we pulled index buffer bytes, then we could - // have a negative value here: + // 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 != null) { + 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)); } } + // If we are using more than 50% of our budget across both indexing buffer and bytes we are moving to disk, then we now + // throttle the top shards to give back-pressure: + boolean doThrottle = (totalBytesWriting + totalBytesUsed) > 1.5 * indexingBuffer.bytes(); + while (totalBytesUsed > indexingBuffer.bytes() && queue.isEmpty() == false) { ShardAndBytesUsed largest = queue.poll(); - System.out.println("IMC: write " + largest.shard.shardId() + ": " + (largest.bytesUsed/1024./1024.) + " MB"); - logger.debug("refresh shard [{}] to free up its [{}] indexing buffer", largest.shard.shardId(), new ByteSizeValue(largest.bytesUsed)); + 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); + largest.shard.activateThrottling(); + } + } + + if (doThrottle == false) { + for(IndexShard shard : throttled) { + logger.info("stop throttling indexing for shard [{}]", shard.shardId()); + shard.deactivateThrottling(); + } + throttled.clear(); } } From 52c092e4fa5964894c1d5f4a40fa987c71cfa960 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 16 Dec 2015 09:18:28 -0500 Subject: [PATCH 10/33] pull out constant for default 5 minute idle setting --- .../java/org/elasticsearch/index/shard/IndexShard.java | 2 +- .../indices/memory/IndexingMemoryController.java | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 3ac6d1ceefa..6e68547d423 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1496,7 +1496,7 @@ public class IndexShard extends AbstractIndexShardComponent { return new EngineConfig(shardId, threadPool, indexingService, indexSettings, engineWarmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig, mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, - idxSettings.getSettings().getAsTime(IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5))); // nocommit + idxSettings.getSettings().getAsTime(IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, IndexingMemoryController.SHARD_DEFAULT_INACTIVE_TIME)); } private static class IndexShardOperationCounter extends AbstractRefCounted { diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index c3199426b13..6e1a0b00393 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -56,6 +56,9 @@ public class IndexingMemoryController extends AbstractLifecycleComponent Date: Wed, 16 Dec 2015 10:22:18 -0500 Subject: [PATCH 11/33] factor out exception handling cases --- .../elasticsearch/index/shard/IndexShard.java | 59 ++++++++----------- 1 file changed, 25 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 6e68547d423..017dcdd6969 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1223,6 +1223,29 @@ public class IndexShard extends AbstractIndexShardComponent { } } + private void handleRefreshException(Exception e) { + if (e instanceof EngineClosedException) { + // ignore + } else if (e instanceof RefreshFailedEngineException e) { + 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. */ @@ -1242,24 +1265,8 @@ public class IndexShard extends AbstractIndexShardComponent { } finally { indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); } - } catch (EngineClosedException ex) { - // 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); } } }); @@ -1282,24 +1289,8 @@ public class IndexShard extends AbstractIndexShardComponent { 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(); From 319dc8c8ed0277116ed8fe3bc866a6f701b800cc Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Wed, 16 Dec 2015 16:19:07 -0500 Subject: [PATCH 12/33] remove dead code; get one test working again; fix docs; remove nocommits --- .../index/CompositeIndexEventListener.java | 12 ---- .../elasticsearch/index/engine/Engine.java | 22 +++--- .../index/engine/EngineConfig.java | 8 +-- .../index/engine/InternalEngine.java | 19 ++--- .../index/shard/IndexEventListener.java | 8 --- .../elasticsearch/index/shard/IndexShard.java | 19 +++-- .../cluster/IndicesClusterStateService.java | 2 +- .../memory/IndexingMemoryController.java | 10 +-- .../elasticsearch/index/IndexModuleTests.java | 1 - .../index/engine/InternalEngineTests.java | 52 -------------- .../memory/IndexingMemoryControllerTests.java | 69 ++++++++++++++++++- .../modules/indices/indexing_buffer.asciidoc | 9 +-- 12 files changed, 98 insertions(+), 133 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java b/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java index e81a4a7c86b..f4ddd75ba85 100644 --- a/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java +++ b/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java @@ -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) { diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 410d2af8d0b..c3dfe6402dd 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -19,17 +19,6 @@ package org.elasticsearch.index.engine; -import java.io.Closeable; -import java.io.IOException; -import java.util.*; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Function; - import org.apache.lucene.index.*; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; @@ -61,6 +50,17 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; +import java.io.Closeable; +import java.io.IOException; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; + /** * */ diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 98572c39f4e..beb93829d93 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -89,11 +89,6 @@ public final class EngineConfig { */ public static final String INDEX_CODEC_SETTING = "index.codec"; - /** - * Index setting to control the index buffer size. - * This setting is not realtime updateable. - */ - /** 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"; @@ -128,7 +123,8 @@ public final class EngineConfig { this.eventListener = eventListener; this.compoundOnFlush = settings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); codecName = settings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME); - // We give IndexWriter a huge buffer, so it won't flush on its own. Instead, IndexingMemoryController periodically checks + // 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(); diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 486920dc750..098868e2b3f 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -103,6 +103,11 @@ public class InternalEngine extends Engine { private final IndexThrottle throttle; + // How many callers are currently requesting index throttling. Currently there are only two times 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(); @@ -516,12 +521,11 @@ public class InternalEngine extends Engine { long versionMapBytes = versionMap.ramBytesUsedForRefresh(); long indexingBufferBytes = indexWriter.ramBytesUsed(); - boolean useRefresh = versionMapRefreshPending.get() || (indexingBufferBytes/4 < versionMapBytes); - // we obtain a read lock here, since we don't want a flush to happen while we are refreshing // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); + 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=[{}])", @@ -542,15 +546,6 @@ public class InternalEngine extends Engine { failEngine("writeIndexingBuffer failed", t); throw new RefreshFailedEngineException(shardId, t); } - - // TODO: maybe we should just put a scheduled job in threadPool? - // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes - // for a long time: - if (useRefresh) { - maybePruneDeletedTombstones(); - versionMapRefreshPending.set(false); - mergeScheduler.refreshConfig(); - } } @Override @@ -1051,8 +1046,6 @@ public class InternalEngine extends Engine { } } - private final AtomicInteger throttleRequestCount = new AtomicInteger(); - @Override public void activateThrottling() { int count = throttleRequestCount.incrementAndGet(); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java index 9a55b9b6161..8d3523a18b1 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 017dcdd6969..4eed36d4424 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -488,7 +488,10 @@ public class IndexShard extends AbstractIndexShardComponent { throw ex; } indexingService.postIndex(index); + + // Notify IMC so that it can go and check heap used by all indexing buffers periodically: indexingMemoryController.bytesWritten(index.getTranslogLocation().size); + return created; } @@ -525,6 +528,8 @@ public class IndexShard extends AbstractIndexShardComponent { throw ex; } indexingService.postDelete(delete); + + // Notify IMC so that it can go and check heap used by all indexing buffers periodically: indexingMemoryController.bytesWritten(delete.getTranslogLocation().size); } @@ -533,13 +538,13 @@ 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(); - // nocommit OK to throw EngineClosedExc? long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); indexingMemoryController.addWritingBytes(this, ramBytesUsed); try { - logger.debug("refresh with source: {} indexBufferRAMBytesUsed={}", source, ramBytesUsed); + logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(ramBytesUsed)); long time = System.nanoTime(); getEngine().refresh(source); refreshMetric.inc(System.nanoTime() - time); @@ -1019,14 +1024,6 @@ public class IndexShard extends AbstractIndexShardComponent { } } - /** - * Returns {@code true} if this shard is active (has seen indexing ops in the last {@link - * IndexingMemoryController#SHARD_INACTIVE_TIME_SETTING} (default 5 minutes), else {@code false}. - */ - public boolean getActive() { - return active.get(); - } - public final boolean isFlushOnClose() { return flushOnClose; } @@ -1226,7 +1223,7 @@ public class IndexShard extends AbstractIndexShardComponent { private void handleRefreshException(Exception e) { if (e instanceof EngineClosedException) { // ignore - } else if (e instanceof RefreshFailedEngineException e) { + } else if (e instanceof RefreshFailedEngineException) { RefreshFailedEngineException rfee = (RefreshFailedEngineException) e; if (rfee.getCause() instanceof InterruptedException) { // ignore, we are being shutdown diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 64ff6c74587..3728628f755 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -119,7 +119,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent implements IndexEventListener { +public class IndexingMemoryController extends AbstractLifecycleComponent { /** 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"; @@ -136,9 +134,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent availableShards() { - return Collections.singletonList(new ShardId("foo", 0)); - } - - @Override - protected void refreshShardAsync(IndexShard shard) { - engine.refresh("memory"); - } - - @Override - protected long getIndexBufferRAMBytesUsed(ShardId shardId) { - System.out.println("BYTES USED: " + engine.indexBufferRAMBytesUsed()); - return engine.indexBufferRAMBytesUsed(); - } - }; - - 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)); - } - - imc.forceCheck(); - try (Searcher s2 = engine.acquireSearcher("test")) { - assertThat(((DirectoryReader) s2.reader()).getVersion(), greaterThan(version1)); - } - } - } - */ - public void testMissingTranslog() throws IOException { // test that we can force start the engine , even if the translog is missing. engine.close(); diff --git a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java index 883e7522466..ae14c0d7a32 100644 --- a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java @@ -18,19 +18,23 @@ */ package org.elasticsearch.indices.memory; +import java.util.*; + +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.indices.IndicesService; import org.elasticsearch.test.ESSingleNodeTestCase; -import java.util.*; - 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 { @@ -183,4 +187,65 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(6, ByteSizeUnit.MB))); } + + // #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 availableShards() { + return Collections.singletonList(shard); + } + + @Override + protected long getIndexBufferRAMBytesUsed(IndexShard shard) { + return shard.getIndexBufferRAMBytesUsed(); + } + }; + + 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); + } + } + }); + } } diff --git a/docs/reference/modules/indices/indexing_buffer.asciidoc b/docs/reference/modules/indices/indexing_buffer.asciidoc index e6485733147..d3aa436c5d4 100644 --- a/docs/reference/modules/indices/indexing_buffer.asciidoc +++ b/docs/reference/modules/indices/indexing_buffer.asciidoc @@ -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`. - - From cbb64634250abb6e6dba5c80ac9aca961f547aae Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Thu, 17 Dec 2015 06:37:52 -0500 Subject: [PATCH 13/33] add throttling test case --- .../elasticsearch/index/shard/IndexShard.java | 37 +++-- .../memory/IndexingMemoryController.java | 53 ++++--- .../memory/IndexingMemoryControllerTests.java | 149 +++++++++++++++--- 3 files changed, 177 insertions(+), 62 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 4eed36d4424..4f4e46c91f3 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -541,15 +541,22 @@ public class IndexShard extends AbstractIndexShardComponent { /** 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(); - long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); - indexingMemoryController.addWritingBytes(this, ramBytesUsed); - try { + if (canIndex()) { + long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); + indexingMemoryController.addWritingBytes(this, ramBytesUsed); logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(ramBytesUsed)); + try { + long time = System.nanoTime(); + getEngine().refresh(source); + refreshMetric.inc(System.nanoTime() - time); + } finally { + indexingMemoryController.removeWritingBytes(this, ramBytesUsed); + } + } else { + logger.debug("refresh with source [{}]", source); long time = System.nanoTime(); getEngine().refresh(source); refreshMetric.inc(System.nanoTime() - time); - } finally { - indexingMemoryController.removeWritingBytes(this, ramBytesUsed); } } @@ -1252,15 +1259,19 @@ public class IndexShard extends AbstractIndexShardComponent { public void run() { try { Engine engine = getEngine(); - long bytes = engine.indexBufferRAMBytesUsed(); - // NOTE: this can be an overestimate by up to 20%, if engine uses IW.flush not refresh, 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: - indexingMemoryController.addWritingBytes(IndexShard.this, bytes); - try { + if (canIndex()) { + long bytes = engine.indexBufferRAMBytesUsed(); + // 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: + indexingMemoryController.addWritingBytes(IndexShard.this, bytes); + try { + getEngine().writeIndexingBuffer(); + } finally { + indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); + } + } else { getEngine().writeIndexingBuffer(); - } finally { - indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); } } catch (Exception e) { handleRefreshException(e); diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index a1c45278c2f..b2239bfb596 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -76,7 +76,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent CAN_UPDATE_INDEX_BUFFER_STATES = EnumSet.of( + private static final EnumSet CAN_WRITE_INDEX_BUFFER_STATES = EnumSet.of( IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED, IndexShardState.RELOCATED); private final ShardsIndicesStatusChecker statusChecker; @@ -129,13 +129,16 @@ public class IndexingMemoryController extends AbstractLifecycleComponent { final long bytesUsed; final IndexShard shard; @@ -273,6 +281,10 @@ public class IndexingMemoryController extends AbstractLifecycleComponent 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: logger.debug("now write some indexing buffers: total indexing heap bytes used [{}] vs {} [{}], currently writing bytes [{}]", @@ -309,10 +321,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent 1.5 * indexingBuffer.bytes(); - 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)); @@ -321,17 +329,17 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexBufferRAMBytesUsed = new HashMap<>(); + // How many bytes this shard is currently moving to disk + final Map writingBytes = new HashMap<>(); + + // Shards that are currently throttled + final Set throttled = new HashSet<>(); + public MockController(Settings settings) { super(Settings.builder() .put(SHARD_MEMORY_INTERVAL_TIME_SETTING, "200h") // disable it @@ -53,6 +60,7 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { public void deleteShard(IndexShard shard) { indexBufferRAMBytesUsed.remove(shard); + writingBytes.remove(shard); } @Override @@ -60,19 +68,9 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { return new ArrayList<>(indexBufferRAMBytesUsed.keySet()); } - @Override - protected boolean shardAvailable(IndexShard shard) { - return indexBufferRAMBytesUsed.containsKey(shard); - } - @Override protected long getIndexBufferRAMBytesUsed(IndexShard shard) { - Long used = indexBufferRAMBytesUsed.get(shard); - if (used == null) { - return 0; - } else { - return used; - } + return indexBufferRAMBytesUsed.get(shard) + writingBytes.get(shard); } @Override @@ -81,18 +79,57 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { @Override public void writeIndexingBufferAsync(IndexShard shard) { + long bytes = indexBufferRAMBytesUsed.put(shard, 0L); + writingBytes.put(shard, writingBytes.get(shard) + bytes); + addWritingBytes(shard, bytes); indexBufferRAMBytesUsed.put(shard, 0L); } - public void assertBuffer(IndexShard shard, ByteSizeValue expected) { + @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) { + long bytes = writingBytes.put(shard, 0L); + removeWritingBytes(shard, bytes); + } + + public void assertBuffer(IndexShard shard, int expectedMB) { Long actual = indexBufferRAMBytesUsed.get(shard); - assertEquals(expected.bytes(), actual.longValue()); + 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) { Long bytes = indexBufferRAMBytesUsed.get(shard); if (bytes == null) { bytes = 0L; + // First time we are seeing this shard: + writingBytes.put(shard, 0L); } // Each doc we index takes up a megabyte! bytes += 1024*1024; @@ -110,18 +147,18 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { .put(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "4mb").build()); IndexShard shard0 = test.getShard(0); controller.simulateIndexing(shard0); - controller.assertBuffer(shard0, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard0, 1); // add another shard IndexShard shard1 = test.getShard(1); controller.simulateIndexing(shard1); - controller.assertBuffer(shard0, new ByteSizeValue(1, ByteSizeUnit.MB)); - controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard0, 1); + controller.assertBuffer(shard1, 1); // remove first shard controller.deleteShard(shard0); controller.forceCheck(); - controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard1, 1); // remove second shard controller.deleteShard(shard1); @@ -130,7 +167,7 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { // add a new one IndexShard shard2 = test.getShard(2); controller.simulateIndexing(shard2); - controller.assertBuffer(shard2, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard2, 1); } public void testActiveInactive() { @@ -148,28 +185,28 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { IndexShard shard1 = test.getShard(1); controller.simulateIndexing(shard1); - controller.assertBuffer(shard0, new ByteSizeValue(1, ByteSizeUnit.MB)); - controller.assertBuffer(shard1, new ByteSizeValue(1, ByteSizeUnit.MB)); + controller.assertBuffer(shard0, 1); + controller.assertBuffer(shard1, 1); controller.simulateIndexing(shard0); controller.simulateIndexing(shard1); - controller.assertBuffer(shard0, new ByteSizeValue(2, ByteSizeUnit.MB)); - controller.assertBuffer(shard1, new ByteSizeValue(2, ByteSizeUnit.MB)); + controller.assertBuffer(shard0, 2); + controller.assertBuffer(shard1, 2); // index into one shard only, crosses the 5mb limit, so shard1 is refreshed controller.simulateIndexing(shard0); controller.simulateIndexing(shard0); - controller.assertBuffer(shard0, new ByteSizeValue(0, ByteSizeUnit.MB)); - controller.assertBuffer(shard1, new ByteSizeValue(2, ByteSizeUnit.MB)); + controller.assertBuffer(shard0, 0); + controller.assertBuffer(shard1, 2); controller.simulateIndexing(shard1); controller.simulateIndexing(shard1); - controller.assertBuffer(shard1, new ByteSizeValue(4, ByteSizeUnit.MB)); + controller.assertBuffer(shard1, 4); controller.simulateIndexing(shard1); controller.simulateIndexing(shard1); // shard1 crossed 5 mb and is now cleared: - controller.assertBuffer(shard1, new ByteSizeValue(0, ByteSizeUnit.MB)); + controller.assertBuffer(shard1, 0); } public void testMinBufferSizes() { @@ -188,6 +225,66 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { assertThat(controller.indexingBufferSize(), equalTo(new ByteSizeValue(6, ByteSizeUnit.MB))); } + 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); + 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.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.assertNotThrottled(shard0); + controller.assertNotThrottled(shard1); + } + // #10312 public void testDeletesAloneCanTriggerRefresh() throws Exception { createIndex("index", From 485f4171bb9797b6dd81237bd0e87b52abe88261 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Thu, 17 Dec 2015 13:28:59 -0500 Subject: [PATCH 14/33] a few cleanups --- .../index/engine/InternalEngine.java | 21 ++++++++------- .../elasticsearch/index/shard/IndexShard.java | 27 +++++++++---------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 098868e2b3f..ed8046a2676 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -103,9 +103,9 @@ public class InternalEngine extends Engine { private final IndexThrottle throttle; - // How many callers are currently requesting index throttling. Currently there are only two times 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: + // 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 { @@ -515,16 +515,17 @@ public class InternalEngine extends Engine { @Override public void writeIndexingBuffer() throws EngineException { - // 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: - long versionMapBytes = versionMap.ramBytesUsedForRefresh(); - long indexingBufferBytes = indexWriter.ramBytesUsed(); - - // we obtain a read lock here, since we don't want a flush to happen while we are refreshing + // 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: + long versionMapBytes = versionMap.ramBytesUsedForRefresh(); + long indexingBufferBytes = indexWriter.ramBytesUsed(); + 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 diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 4f4e46c91f3..da7ebfb4ad8 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -544,8 +544,8 @@ public class IndexShard extends AbstractIndexShardComponent { if (canIndex()) { long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); indexingMemoryController.addWritingBytes(this, ramBytesUsed); - logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(ramBytesUsed)); try { + logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(ramBytesUsed)); long time = System.nanoTime(); getEngine().refresh(source); refreshMetric.inc(System.nanoTime() - time); @@ -1019,7 +1019,7 @@ public class IndexShard extends AbstractIndexShardComponent { } /** 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. */ + * 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) { @@ -1254,24 +1254,23 @@ public class IndexShard extends AbstractIndexShardComponent { * Called when our shard is using too much heap and should move buffered indexed/deleted documents to disk. */ public void writeIndexingBufferAsync() { + if (canIndex() == false) { + throw new UnsupportedOperationException(); + } threadPool.executor(ThreadPool.Names.REFRESH).execute(new Runnable() { @Override public void run() { try { Engine engine = getEngine(); - if (canIndex()) { - long bytes = engine.indexBufferRAMBytesUsed(); - // 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: - indexingMemoryController.addWritingBytes(IndexShard.this, bytes); - try { - getEngine().writeIndexingBuffer(); - } finally { - indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); - } - } else { + long bytes = engine.indexBufferRAMBytesUsed(); + // 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: + indexingMemoryController.addWritingBytes(IndexShard.this, bytes); + try { getEngine().writeIndexingBuffer(); + } finally { + indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); } } catch (Exception e) { handleRefreshException(e); From 99d6ec53faa2043b1486253ce9cf9049bed67eb2 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Tue, 5 Jan 2016 09:53:13 -0500 Subject: [PATCH 15/33] fold in feedback --- .../elasticsearch/index/engine/Engine.java | 2 +- .../index/engine/InternalEngine.java | 14 +-- .../index/engine/ShadowEngine.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 54 ++++++---- .../memory/IndexingMemoryController.java | 101 +++++++++--------- .../memory/IndexingMemoryControllerTests.java | 15 ++- 6 files changed, 105 insertions(+), 83 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index c3dfe6402dd..212bfe309af 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -375,7 +375,7 @@ public abstract class Engine implements Closeable { } /** How much heap is used that would be freed by a refresh. Note that this may throw {@link AlreadyClosedException}. */ - abstract public long indexBufferRAMBytesUsed(); + abstract public long getIndexBufferRAMBytesUsed(); protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { ensureOpen(); diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index ed8046a2676..39d898f115a 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -522,11 +522,11 @@ public class InternalEngine extends Engine { // 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: - long versionMapBytes = versionMap.ramBytesUsedForRefresh(); - long indexingBufferBytes = indexWriter.ramBytesUsed(); + // refresh API), and another for version map interactions. See #15768. + final long versionMapBytes = versionMap.ramBytesUsedForRefresh(); + final long indexingBufferBytes = indexWriter.ramBytesUsed(); - boolean useRefresh = versionMapRefreshPending.get() || (indexingBufferBytes/4 < versionMapBytes); + 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=[{}])", @@ -823,7 +823,7 @@ public class InternalEngine extends Engine { } @Override - public long indexBufferRAMBytesUsed() { + public long getIndexBufferRAMBytesUsed() { return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); } @@ -1050,7 +1050,7 @@ public class InternalEngine extends Engine { @Override public void activateThrottling() { int count = throttleRequestCount.incrementAndGet(); - assert count >= 1; + assert count >= 1: "invalid post-increment throttleRequestCount=" + count; if (count == 1) { throttle.activate(); } @@ -1059,7 +1059,7 @@ public class InternalEngine extends Engine { @Override public void deactivateThrottling() { int count = throttleRequestCount.decrementAndGet(); - assert count >= 0; + assert count >= 0: "invalid post-decrement throttleRequestCount=" + count; if (count == 0) { throttle.deactivate(); } diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 286b06e69f9..49fd7328000 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -227,7 +227,7 @@ public class ShadowEngine extends Engine { } @Override - public long indexBufferRAMBytesUsed() { + public long getIndexBufferRAMBytesUsed() { // No IndexWriter nor version map throw new UnsupportedOperationException("ShadowEngine has no IndexWriter"); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index da7ebfb4ad8..52f01ee08b1 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -19,6 +19,18 @@ package org.elasticsearch.index.shard; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.channels.ClosedByInterruptException; +import java.nio.charset.StandardCharsets; +import java.util.*; +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; + import org.apache.lucene.index.*; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.UsageTrackingQueryCachingPolicy; @@ -81,8 +93,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; @@ -104,17 +116,6 @@ import org.elasticsearch.search.suggest.completion.CompletionFieldStats; import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.threadpool.ThreadPool; -import java.io.IOException; -import java.io.PrintStream; -import java.nio.channels.ClosedByInterruptException; -import java.nio.charset.StandardCharsets; -import java.util.*; -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.AtomicReference; - public class IndexShard extends AbstractIndexShardComponent { @@ -148,6 +149,9 @@ public class IndexShard extends AbstractIndexShardComponent { private final IndexSettings idxSettings; private final NodeServicesProvider provider; + /** How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh */ + private final AtomicLong writingBytes = new AtomicLong(); + private TimeValue refreshInterval; private volatile ScheduledFuture refreshScheduledFuture; @@ -542,15 +546,17 @@ public class IndexShard extends AbstractIndexShardComponent { public void refresh(String source) { verifyNotClosed(); if (canIndex()) { - long ramBytesUsed = getEngine().indexBufferRAMBytesUsed(); - indexingMemoryController.addWritingBytes(this, ramBytesUsed); + long bytes = getEngine().getIndexBufferRAMBytesUsed(); + writingBytes.addAndGet(bytes); try { - logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(ramBytesUsed)); + logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(bytes)); long time = System.nanoTime(); getEngine().refresh(source); refreshMetric.inc(System.nanoTime() - time); } finally { - indexingMemoryController.removeWritingBytes(this, ramBytesUsed); + logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); + // nocommit but we don't promptly stop index throttling anymore? + writingBytes.addAndGet(-bytes); } } else { logger.debug("refresh with source [{}]", source); @@ -560,6 +566,11 @@ public class IndexShard extends AbstractIndexShardComponent { } } + /** Returns how many bytes we are currently moving from heap to disk */ + public long getWritingBytes() { + return writingBytes.get(); + } + public RefreshStats refreshStats() { return new RefreshStats(refreshMetric.count(), TimeUnit.NANOSECONDS.toMillis(refreshMetric.sum())); } @@ -1008,7 +1019,7 @@ public class IndexShard extends AbstractIndexShardComponent { return 0; } try { - return engine.indexBufferRAMBytesUsed(); + return engine.getIndexBufferRAMBytesUsed(); } catch (AlreadyClosedException ex) { return 0; } @@ -1262,15 +1273,18 @@ public class IndexShard extends AbstractIndexShardComponent { public void run() { try { Engine engine = getEngine(); - long bytes = engine.indexBufferRAMBytesUsed(); + 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: - indexingMemoryController.addWritingBytes(IndexShard.this, bytes); + writingBytes.addAndGet(bytes); + logger.debug("add [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); try { getEngine().writeIndexingBuffer(); } finally { - indexingMemoryController.removeWritingBytes(IndexShard.this, bytes); + logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); + // nocommit but we don't promptly stop index throttling anymore? + writingBytes.addAndGet(-bytes); } } catch (Exception e) { handleRefreshException(e); diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index b2239bfb596..9b0e538855e 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -23,6 +23,7 @@ import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; @@ -81,9 +82,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent writingBytes = new ConcurrentHashMap<>(); - @Inject public IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) { this(settings, threadPool, indicesService, JvmInfo.jvmInfo().getMem().getHeapMax().bytes()); @@ -126,21 +124,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()/30) { - // 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: - run(); + public void bytesWritten(int bytes) { + long totalBytes = bytesWrittenSinceCheck.addAndGet(bytes); + if (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: + run(); + } finally { + runLock.unlock(); + } + } } } @Override - public synchronized void run() { + 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 @@ -257,20 +262,18 @@ public class IndexingMemoryController extends AbstractLifecycleComponent 0) { if (logger.isTraceEnabled()) { - if (shardWritingBytes != null) { + 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); @@ -341,8 +342,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent Date: Tue, 5 Jan 2016 11:27:05 -0500 Subject: [PATCH 16/33] move async-ness upwards --- .../elasticsearch/index/shard/IndexShard.java | 44 +++++++++---------- .../memory/IndexingMemoryController.java | 7 ++- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 52f01ee08b1..a34a441ae8b 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1264,33 +1264,29 @@ public class IndexShard extends AbstractIndexShardComponent { /** * Called when our shard is using too much heap and should move buffered indexed/deleted documents to disk. */ - public void writeIndexingBufferAsync() { + public void writeIndexingBuffer() { if (canIndex() == false) { throw new UnsupportedOperationException(); } - threadPool.executor(ThreadPool.Names.REFRESH).execute(new Runnable() { - @Override - public void run() { - 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: - writingBytes.addAndGet(bytes); - logger.debug("add [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); - try { - getEngine().writeIndexingBuffer(); - } finally { - logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); - // nocommit but we don't promptly stop index throttling anymore? - writingBytes.addAndGet(-bytes); - } - } catch (Exception e) { - handleRefreshException(e); - } - } - }); + 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 { + // nocommit but we don't promptly stop index throttling anymore? + writingBytes.addAndGet(-bytes); + logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); + } + } catch (Exception e) { + handleRefreshException(e); + }; } final class EngineRefresher implements Runnable { diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 9b0e538855e..f63534e1ac0 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -174,7 +174,12 @@ public class IndexingMemoryController extends AbstractLifecycleComponent Date: Wed, 6 Jan 2016 03:41:23 -0500 Subject: [PATCH 17/33] remove nocommits; fix test case --- .../java/org/elasticsearch/index/shard/IndexShard.java | 2 -- .../indices/memory/IndexingMemoryControllerTests.java | 7 +++++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index a34a441ae8b..049391e48de 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -555,7 +555,6 @@ public class IndexShard extends AbstractIndexShardComponent { refreshMetric.inc(System.nanoTime() - time); } finally { logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); - // nocommit but we don't promptly stop index throttling anymore? writingBytes.addAndGet(-bytes); } } else { @@ -1280,7 +1279,6 @@ public class IndexShard extends AbstractIndexShardComponent { try { engine.writeIndexingBuffer(); } finally { - // nocommit but we don't promptly stop index throttling anymore? writingBytes.addAndGet(-bytes); logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId()); } diff --git a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java index 64259c7be97..3fcf2467ca8 100644 --- a/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/memory/IndexingMemoryControllerTests.java @@ -330,7 +330,14 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { @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(); } + }; for (int i = 0; i < 100; i++) { From 81a76072565412dab63e8ed0260c716b7600cdeb Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 7 Jan 2016 09:28:44 -0500 Subject: [PATCH 18/33] Remove -Xlint:-deprecation from plugins Instead we suppress warnings about using deprecated stuff near the usage site with a comment about why its ok. --- modules/lang-groovy/build.gradle | 4 +- plugins/analysis-icu/build.gradle | 3 -- .../IcuCollationTokenFilterFactory.java | 15 ++++--- plugins/discovery-azure/build.gradle | 2 - plugins/discovery-ec2/build.gradle | 2 - .../cloud/aws/AwsEc2ServiceImpl.java | 5 ++- .../discovery/ec2/Ec2DiscoveryPlugin.java | 13 +++--- plugins/discovery-multicast/build.gradle | 2 - .../discovery/multicast/MulticastZenPing.java | 41 +++++++++++-------- plugins/mapper-attachments/build.gradle | 2 +- .../mapper/attachments/AttachmentMapper.java | 5 ++- plugins/repository-azure/build.gradle | 3 +- .../azure/storage/AzureStorageSettings.java | 7 ++-- ...bstractAzureRepositoryServiceTestCase.java | 10 +++-- .../azure/AzureSettingsParserTest.java | 5 ++- plugins/repository-hdfs/build.gradle | 2 - plugins/repository-s3/build.gradle | 2 - .../cloud/aws/InternalAwsS3Service.java | 3 +- .../repository/s3/S3RepositoryPlugin.java | 1 + 19 files changed, 68 insertions(+), 59 deletions(-) diff --git a/modules/lang-groovy/build.gradle b/modules/lang-groovy/build.gradle index 7db4eab4403..73ad6043f37 100644 --- a/modules/lang-groovy/build.gradle +++ b/modules/lang-groovy/build.gradle @@ -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 { diff --git a/plugins/analysis-icu/build.gradle b/plugins/analysis-icu/build.gradle index a662f727eeb..9ed155b5fc4 100644 --- a/plugins/analysis-icu/build.gradle +++ b/plugins/analysis-icu/build.gradle @@ -30,6 +30,3 @@ dependencies { dependencyLicenses { mapping from: /lucene-.*/, to: 'lucene' } - -compileJava.options.compilerArgs << "-Xlint:-deprecation" - diff --git a/plugins/analysis-icu/src/main/java/org/elasticsearch/index/analysis/IcuCollationTokenFilterFactory.java b/plugins/analysis-icu/src/main/java/org/elasticsearch/index/analysis/IcuCollationTokenFilterFactory.java index 1e7cd1b09f4..a8b71f3f2f9 100644 --- a/plugins/analysis-icu/src/main/java/org/elasticsearch/index/analysis/IcuCollationTokenFilterFactory.java +++ b/plugins/analysis-icu/src/main/java/org/elasticsearch/index/analysis/IcuCollationTokenFilterFactory.java @@ -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); } diff --git a/plugins/discovery-azure/build.gradle b/plugins/discovery-azure/build.gradle index 12b479eb487..f6b56c9823a 100644 --- a/plugins/discovery-azure/build.gradle +++ b/plugins/discovery-azure/build.gradle @@ -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 diff --git a/plugins/discovery-ec2/build.gradle b/plugins/discovery-ec2/build.gradle index 403b2638257..e8baa1ec502 100644 --- a/plugins/discovery-ec2/build.gradle +++ b/plugins/discovery-ec2/build.gradle @@ -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 diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java index 349a513455c..40dc8409dd4 100644 --- a/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java +++ b/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java @@ -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,8 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent discoveryNodeService.addCustomAttributeProvider(new Ec2CustomNodeAttributes(settings)); } + @Override + @SuppressWarnings("deprecation") // Supports deprecated parameters for backwards compatibility public synchronized AmazonEC2 client() { if (client != null) { return client; @@ -135,7 +138,7 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent 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, diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/plugin/discovery/ec2/Ec2DiscoveryPlugin.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/plugin/discovery/ec2/Ec2DiscoveryPlugin.java index ffa76c6b9b3..3b3d206c255 100644 --- a/plugins/discovery-ec2/src/main/java/org/elasticsearch/plugin/discovery/ec2/Ec2DiscoveryPlugin.java +++ b/plugins/discovery-ec2/src/main/java/org/elasticsearch/plugin/discovery/ec2/Ec2DiscoveryPlugin.java @@ -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> nodeServices() { Collection> services = new ArrayList<>(); services.add(AwsEc2ServiceImpl.class); diff --git a/plugins/discovery-multicast/build.gradle b/plugins/discovery-multicast/build.gradle index f48f62841b7..295f28c094b 100644 --- a/plugins/discovery-multicast/build.gradle +++ b/plugins/discovery-multicast/build.gradle @@ -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" diff --git a/plugins/discovery-multicast/src/main/java/org/elasticsearch/plugin/discovery/multicast/MulticastZenPing.java b/plugins/discovery-multicast/src/main/java/org/elasticsearch/plugin/discovery/multicast/MulticastZenPing.java index 82bf1bf088c..81f8ed28f84 100644 --- a/plugins/discovery-multicast/src/main/java/org/elasticsearch/plugin/discovery/multicast/MulticastZenPing.java +++ b/plugins/discovery-multicast/src/main/java/org/elasticsearch/plugin/discovery/multicast/MulticastZenPing.java @@ -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 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 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 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 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) { diff --git a/plugins/mapper-attachments/build.gradle b/plugins/mapper-attachments/build.gradle index bbe89aa1fd4..70741c2094d 100644 --- a/plugins/mapper-attachments/build.gradle +++ b/plugins/mapper-attachments/build.gradle @@ -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' diff --git a/plugins/mapper-attachments/src/main/java/org/elasticsearch/mapper/attachments/AttachmentMapper.java b/plugins/mapper-attachments/src/main/java/org/elasticsearch/mapper/attachments/AttachmentMapper.java index d43b5df7e4a..7c54e6f17ce 100644 --- a/plugins/mapper-attachments/src/main/java/org/elasticsearch/mapper/attachments/AttachmentMapper.java +++ b/plugins/mapper-attachments/src/main/java/org/elasticsearch/mapper/attachments/AttachmentMapper.java @@ -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) 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; diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index f0c21bb3828..85a927c02f9 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -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' diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java b/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java index c7380e2fd7f..fdc0f40b27b 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java @@ -19,6 +19,9 @@ package org.elasticsearch.cloud.azure.storage; +import java.util.HashMap; +import java.util.Map; + import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.ESLogger; @@ -28,9 +31,6 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.repositories.RepositorySettings; -import java.util.HashMap; -import java.util.Map; - public class AzureStorageSettings { private static ESLogger logger = ESLoggerFactory.getLogger(AzureStorageSettings.class.getName()); @@ -78,6 +78,7 @@ public class AzureStorageSettings { * @param settings settings to parse * @return A tuple with v1 = primary storage and v2 = secondary storage */ + @SuppressWarnings("deprecation") // Supports deprecated settings public static Tuple> parse(Settings settings) { AzureStorageSettings primaryStorage = null; Map secondaryStorage = new HashMap<>(); diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java index b3e878927e9..43714e4d538 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java @@ -19,8 +19,9 @@ package org.elasticsearch.cloud.azure; -import com.microsoft.azure.storage.LocationMode; -import com.microsoft.azure.storage.StorageException; +import java.net.URISyntaxException; +import java.util.Collection; + import org.elasticsearch.cloud.azure.storage.AzureStorageService; import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage; import org.elasticsearch.cloud.azure.storage.AzureStorageServiceMock; @@ -33,8 +34,8 @@ import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.After; import org.junit.Before; -import java.net.URISyntaxException; -import java.util.Collection; +import com.microsoft.azure.storage.LocationMode; +import com.microsoft.azure.storage.StorageException; public abstract class AbstractAzureRepositoryServiceTestCase extends AbstractAzureTestCase { @@ -76,6 +77,7 @@ public abstract class AbstractAzureRepositoryServiceTestCase extends AbstractAzu } @Override + @SuppressWarnings("deprecation") // Supports deprecated settings for testing backwards compatibility protected Settings nodeSettings(int nodeOrdinal) { Settings.Builder builder = Settings.settingsBuilder() .put(Storage.CONTAINER, "snapshots"); diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java index 59e8b8945c0..8cb9262af85 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java @@ -19,14 +19,14 @@ package org.elasticsearch.repositories.azure; +import java.util.Map; + 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; -import java.util.Map; - import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -67,6 +67,7 @@ public class AzureSettingsParserTest extends LuceneTestCase { } public void testDeprecatedSettings() { + @SuppressWarnings("deprecation") Settings settings = Settings.builder() .put(Storage.ACCOUNT_DEPRECATED, "myaccount1") .put(Storage.KEY_DEPRECATED, "mykey1") diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index 68ab6f56ddb..924f8cd1d42 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -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', diff --git a/plugins/repository-s3/build.gradle b/plugins/repository-s3/build.gradle index b11aa732b25..a083309891e 100644 --- a/plugins/repository-s3/build.gradle +++ b/plugins/repository-s3/build.gradle @@ -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 diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java b/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java index 51594c01302..fc26d3e6b71 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java @@ -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,7 @@ public class InternalAwsS3Service extends AbstractLifecycleComponent clientDescriptor = new Tuple(endpoint, account); AmazonS3Client client = clients.get(clientDescriptor); diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/plugin/repository/s3/S3RepositoryPlugin.java b/plugins/repository-s3/src/main/java/org/elasticsearch/plugin/repository/s3/S3RepositoryPlugin.java index a7305727353..2b486e5505c 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/plugin/repository/s3/S3RepositoryPlugin.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/plugin/repository/s3/S3RepositoryPlugin.java @@ -77,6 +77,7 @@ public class S3RepositoryPlugin extends Plugin { } @Override + @SuppressWarnings("rawtypes") // Supertype declaration has raw types public Collection> nodeServices() { return Collections.>singleton(S3Module.getS3ServiceImpl()); } From 00edc8bc87289014f24c218dbc884f719a3d2443 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 7 Jan 2016 21:22:20 -0500 Subject: [PATCH 19/33] Remove -Xlint:-deprecated from test/framwork There were a few uses of deprecation I couldn't clear up quickly so I filed issues for them and suppressed the warnings. --- test/framework/build.gradle | 2 +- .../elasticsearch/common/io/PathUtilsForTesting.java | 11 ++++++----- .../org/elasticsearch/test/InternalTestCluster.java | 4 +++- .../test/disruption/LongGCDisruption.java | 2 ++ .../test/hamcrest/ElasticsearchAssertions.java | 6 +++--- .../test/store/MockFSDirectoryService.java | 2 ++ 6 files changed, 17 insertions(+), 10 deletions(-) diff --git a/test/framework/build.gradle b/test/framework/build.gradle index 5c607e1aa61..46728d06283 100644 --- a/test/framework/build.gradle +++ b/test/framework/build.gradle @@ -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 diff --git a/test/framework/src/main/java/org/elasticsearch/common/io/PathUtilsForTesting.java b/test/framework/src/main/java/org/elasticsearch/common/io/PathUtilsForTesting.java index fee053eed45..36f766b8067 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/io/PathUtilsForTesting.java +++ b/test/framework/src/main/java/org/elasticsearch/common/io/PathUtilsForTesting.java @@ -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; diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index ea2796aad84..81dbc387a14 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -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)) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/disruption/LongGCDisruption.java b/test/framework/src/main/java/org/elasticsearch/test/disruption/LongGCDisruption.java index 1e8dcb10db7..591540e72b9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/disruption/LongGCDisruption.java +++ b/test/framework/src/main/java/org/elasticsearch/test/disruption/LongGCDisruption.java @@ -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 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 threads) { for (Thread thread : threads) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java index 9d8ad7f7dcf..61755f7ecb4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java +++ b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java @@ -513,9 +513,9 @@ public class ElasticsearchAssertions { public static T assertBooleanSubQuery(Query query, Class 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()); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java index 58a72789f65..a5fb865238e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -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; @@ -105,6 +106,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"); From 98fdb39d3d42a12df7c865fa232775e4a0105c38 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 8 Jan 2016 11:17:56 -0500 Subject: [PATCH 20/33] Remove deprecated settings --- .../java/org/elasticsearch/cloud/aws/AwsEc2Service.java | 9 +-------- .../org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java | 9 ++++----- .../java/org/elasticsearch/cloud/aws/AwsS3Service.java | 9 +-------- .../elasticsearch/cloud/aws/InternalAwsS3Service.java | 9 ++++----- 4 files changed, 10 insertions(+), 26 deletions(-) diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2Service.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2Service.java index d71d9dfb0af..e97dd94137b 100644 --- a/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2Service.java +++ b/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2Service.java @@ -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 { @@ -33,10 +34,6 @@ public interface AwsEc2Service extends LifecycleComponent { 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 { 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 { diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java index 40dc8409dd4..48309457d73 100644 --- a/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java +++ b/plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/AwsEc2ServiceImpl.java @@ -73,7 +73,6 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent } @Override - @SuppressWarnings("deprecation") // Supports deprecated parameters for backwards compatibility public synchronized AmazonEC2 client() { if (client != null) { return client; @@ -94,11 +93,11 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent 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); diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/AwsS3Service.java b/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/AwsS3Service.java index 711b8db9374..55c4b58e6dd 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/AwsS3Service.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/AwsS3Service.java @@ -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 { 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 { 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 { diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java b/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java index fc26d3e6b71..90b79fd3b4d 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/cloud/aws/InternalAwsS3Service.java @@ -95,7 +95,6 @@ public class InternalAwsS3Service extends AbstractLifecycleComponent clientDescriptor = new Tuple(endpoint, account); AmazonS3Client client = clients.get(clientDescriptor); @@ -120,11 +119,11 @@ public class InternalAwsS3Service extends AbstractLifecycleComponent Date: Fri, 8 Jan 2016 13:13:14 -0500 Subject: [PATCH 21/33] Remove deprecated azure settings --- .../azure/storage/AzureStorageService.java | 5 +- .../azure/storage/AzureStorageSettings.java | 73 ++++++++----------- ...bstractAzureRepositoryServiceTestCase.java | 14 +--- .../azure/AzureSettingsParserTest.java | 19 +---- 4 files changed, 38 insertions(+), 73 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageService.java b/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageService.java index 9ed909c0b8f..c154f78eeb5 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageService.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageService.java @@ -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"; diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java b/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java index fdc0f40b27b..75414f0d865 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/storage/AzureStorageSettings.java @@ -19,9 +19,6 @@ package org.elasticsearch.cloud.azure.storage; -import java.util.HashMap; -import java.util.Map; - import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.ESLogger; @@ -31,6 +28,9 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.repositories.RepositorySettings; +import java.util.HashMap; +import java.util.Map; + public class AzureStorageSettings { private static ESLogger logger = ESLoggerFactory.getLogger(AzureStorageSettings.class.getName()); @@ -78,57 +78,46 @@ public class AzureStorageSettings { * @param settings settings to parse * @return A tuple with v1 = primary storage and v2 = secondary storage */ - @SuppressWarnings("deprecation") // Supports deprecated settings public static Tuple> parse(Settings settings) { AzureStorageSettings primaryStorage = null; Map 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 asMap = storageSettings.getAsStructuredMap(); - for (Map.Entry storage : asMap.entrySet()) { - if (storage.getValue() instanceof Map) { - @SuppressWarnings("unchecked") - Map 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 asMap = storageSettings.getAsStructuredMap(); + for (Map.Entry storage : asMap.entrySet()) { + if (storage.getValue() instanceof Map) { + @SuppressWarnings("unchecked") + Map 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 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 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()); } } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java index 43714e4d538..3c32adeae4f 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceTestCase.java @@ -19,8 +19,8 @@ package org.elasticsearch.cloud.azure; -import java.net.URISyntaxException; -import java.util.Collection; +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; @@ -34,8 +34,8 @@ import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.After; import org.junit.Before; -import com.microsoft.azure.storage.LocationMode; -import com.microsoft.azure.storage.StorageException; +import java.net.URISyntaxException; +import java.util.Collection; public abstract class AbstractAzureRepositoryServiceTestCase extends AbstractAzureTestCase { @@ -77,15 +77,9 @@ public abstract class AbstractAzureRepositoryServiceTestCase extends AbstractAzu } @Override - @SuppressWarnings("deprecation") // Supports deprecated settings for testing backwards compatibility 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(); } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java index 8cb9262af85..aec8506ca6d 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSettingsParserTest.java @@ -19,14 +19,13 @@ package org.elasticsearch.repositories.azure; -import java.util.Map; - 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; +import java.util.Map; + import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -66,20 +65,6 @@ public class AzureSettingsParserTest extends LuceneTestCase { assertThat(tuple.v2().keySet(), hasSize(0)); } - public void testDeprecatedSettings() { - @SuppressWarnings("deprecation") - Settings settings = Settings.builder() - .put(Storage.ACCOUNT_DEPRECATED, "myaccount1") - .put(Storage.KEY_DEPRECATED, "mykey1") - .build(); - - Tuple> 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") From 6bb01984b6cceab38d50dcb92ba8da5c6a5f28ab Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 8 Jan 2016 16:53:40 -0500 Subject: [PATCH 22/33] Fix blended terms for non-strings It had some funky errors, like lenient:true not working and queries with two integer fields blowing up if there was no analyzer defined on the query. This throws a bunch more tests at it and rejiggers how non-strings are handled so they don't wander off into scary QueryBuilder-land unless they have a nice strong analyzer to protect them. Closes #15860 --- .../index/mapper/MappedFieldType.java | 2 +- .../index/search/MatchQuery.java | 27 +++++-- .../index/search/MultiMatchQuery.java | 48 +++++++++++- .../search/query/MultiMatchQueryIT.java | 76 +++++++++++++++++++ 4 files changed, 142 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java index 5f8049b55fb..28590c56705 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java @@ -390,7 +390,7 @@ public abstract class MappedFieldType extends FieldType { } /** Creates a term associated with the field of this mapper for the given value */ - protected Term createTerm(Object value) { + public Term createTerm(Object value) { return new Term(name(), indexedValueForSearch(value)); } diff --git a/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java b/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java index 1b213645ae5..46dc9d3486a 100644 --- a/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java +++ b/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java @@ -212,10 +212,6 @@ public class MatchQuery { this.zeroTermsQuery = zeroTermsQuery; } - protected boolean forceAnalyzeQueryString() { - return false; - } - protected Analyzer getAnalyzer(MappedFieldType fieldType) { if (this.analyzer == null) { if (fieldType != null) { @@ -240,9 +236,18 @@ public class MatchQuery { field = fieldName; } - if (fieldType != null && fieldType.useTermQueryWithQueryString() && !forceAnalyzeQueryString()) { + /* + * If the user forced an analyzer we really don't care if they are + * searching a type that wants term queries to be used with query string + * because the QueryBuilder will take care of it. If they haven't forced + * an analyzer then types like NumberFieldType that want terms with + * query string will blow up because their analyzer isn't capable of + * passing through QueryBuilder. + */ + boolean noForcedAnalyzer = this.analyzer == null; + if (fieldType != null && fieldType.useTermQueryWithQueryString() && noForcedAnalyzer) { try { - return fieldType.termQuery(value, context); + return termQuery(fieldType, value); } catch (RuntimeException e) { if (lenient) { return null; @@ -251,6 +256,7 @@ public class MatchQuery { } } + Analyzer analyzer = getAnalyzer(fieldType); assert analyzer != null; MatchQueryBuilder builder = new MatchQueryBuilder(analyzer, fieldType); @@ -282,6 +288,15 @@ public class MatchQuery { } } + /** + * Creates a TermQuery-like-query for MappedFieldTypes that don't support + * QueryBuilder which is very string-ish. Just delegates to the + * MappedFieldType for MatchQuery but gets more complex for blended queries. + */ + protected Query termQuery(MappedFieldType fieldType, Object value) { + return fieldType.termQuery(value, context); + } + protected Query zeroTermsQuery() { return zeroTermsQuery == DEFAULT_ZERO_TERMS_QUERY ? Queries.newMatchNoDocsQuery() : Queries.newMatchAllQuery(); } diff --git a/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java b/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java index cf30c3dbe47..e6753538939 100644 --- a/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java +++ b/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java @@ -149,6 +149,10 @@ public class MultiMatchQuery extends MatchQuery { public boolean forceAnalyzeQueryString() { return false; } + + public Query termQuery(MappedFieldType fieldType, Object value) { + return fieldType.termQuery(value, context); + } } public class CrossFieldsQueryBuilder extends QueryBuilder { @@ -196,8 +200,13 @@ public class MultiMatchQuery extends MatchQuery { } else { blendedFields = null; } - final FieldAndFieldType fieldAndFieldType = group.get(0); - Query q = parseGroup(type.matchQueryType(), fieldAndFieldType.field, 1f, value, minimumShouldMatch); + /* + * We have to pick some field to pass through the superclass so + * we just pick the first field. It shouldn't matter because + * fields are already grouped by their analyzers/types. + */ + String representativeField = group.get(0).field; + Query q = parseGroup(type.matchQueryType(), representativeField, 1f, value, minimumShouldMatch); if (q != null) { queries.add(q); } @@ -206,6 +215,28 @@ public class MultiMatchQuery extends MatchQuery { return queries.isEmpty() ? null : queries; } + /** + * Pick the field for parsing. If any of the fields in the group do + * *not* useTermQueryWithQueryString then we return that one to force + * analysis. If some of the fields would useTermQueryWithQueryString + * then we assume that that parsing field's parser is good enough for + * them and return it. Otherwise we just return the first field. You + * should only get mixed groups like this when you force a certain + * analyzer on a query and use string and integer fields because of the + * way that grouping is done. That means that the use *asked* for the + * integer fields to be searched using a string analyzer so this is + * technically doing exactly what they asked for even if it is a bit + * funky. + */ + private String fieldForParsing(List group) { + for (FieldAndFieldType field: group) { + if (field.fieldType.useTermQueryWithQueryString()) { + return field.field; + } + } + return group.get(0).field; + } + @Override public boolean forceAnalyzeQueryString() { return blendedFields != null; @@ -231,6 +262,11 @@ public class MultiMatchQuery extends MatchQuery { } return BlendedTermQuery.dismaxBlendedQuery(terms, blendedBoost, tieBreaker); } + + @Override + public Query termQuery(MappedFieldType fieldType, Object value) { + return blendTerm(fieldType.createTerm(value), fieldType); + } } @Override @@ -266,7 +302,11 @@ public class MultiMatchQuery extends MatchQuery { } @Override - protected boolean forceAnalyzeQueryString() { - return this.queryBuilder == null ? super.forceAnalyzeQueryString() : this.queryBuilder.forceAnalyzeQueryString(); + protected Query termQuery(MappedFieldType fieldType, Object value) { + if (queryBuilder == null) { + // Can be null when the MultiMatchQuery collapses into a MatchQuery + return super.termQuery(fieldType, value); + } + return queryBuilder.termQuery(fieldType, value); } } diff --git a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java index a789497235e..1e070f325b3 100644 --- a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.query; import com.carrotsearch.randomizedtesting.generators.RandomPicks; + import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; @@ -459,6 +460,23 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertHitCount(searchResponse, 1l); assertFirstHit(searchResponse, hasId("theone")); + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("captain america 15", "full_name", "first_name", "last_name", "category", "skill", "int-field") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .analyzer("category") + .operator(Operator.AND))).get(); + assertHitCount(searchResponse, 1l); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("captain america 15", "skill", "full_name", "first_name", "last_name", "category", "int-field") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .analyzer("category") + .operator(Operator.AND))).get(); + assertHitCount(searchResponse, 1l); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("captain america 15", "first_name", "last_name", "skill") .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) @@ -471,6 +489,24 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .analyzer("category"))).get(); assertFirstHit(searchResponse, hasId("theone")); + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("25 15", "first_name", "int-field", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .analyzer("category"))).get(); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("25 15", "int-field", "skill", "first_name") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .analyzer("category"))).get(); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("25 15", "int-field", "first_name", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .analyzer("category"))).get(); + assertFirstHit(searchResponse, hasId("theone")); + searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("captain america marvel hero", "first_name", "last_name", "category") .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) @@ -529,6 +565,46 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertFirstHit(searchResponse, hasId("ultimate2")); assertSecondHit(searchResponse, hasId("ultimate1")); assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + + // Test group based on numeric fields + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("15", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("15", "skill", "first_name") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); + assertFirstHit(searchResponse, hasId("theone")); + + // Two numeric fields together caused trouble at one point! + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("15", "int-field", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("15", "int-field", "first_name", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); + assertFirstHit(searchResponse, hasId("theone")); + + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("alpha 15", "first_name", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .lenient(true))).get(); + assertFirstHit(searchResponse, hasId("ultimate1")); + /* + * Doesn't find theone because "alpha 15" isn't a number and we don't + * break on spaces. + */ + assertHitCount(searchResponse, 1); + + // Lenient wasn't always properly lenient with two numeric fields + searchResponse = client().prepareSearch("test") + .setQuery(randomizeType(multiMatchQuery("alpha 15", "int-field", "first_name", "skill") + .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) + .lenient(true))).get(); + assertFirstHit(searchResponse, hasId("ultimate1")); } private static final void assertEquivalent(String query, SearchResponse left, SearchResponse right) { From d8526f4d003bb58625df921af36941359daea98d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 7 Jan 2016 22:45:05 -0500 Subject: [PATCH 23/33] Handle some deprecation warnings Suppress lots of them with comments. Fix a few. Removes lots of pre-built stuff used to support pre-2.0 indexes. --- .../index/analysis/NGramTokenizerFactory.java | 30 +++----- .../index/query/QueryBuilders.java | 4 +- .../org/elasticsearch/index/store/Store.java | 15 ++-- .../indices/analysis/PreBuiltAnalyzers.java | 21 ++---- .../analysis/PreBuiltTokenFilters.java | 69 +++---------------- .../indices/analysis/PreBuiltTokenizers.java | 34 ++------- 6 files changed, 38 insertions(+), 135 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/analysis/NGramTokenizerFactory.java b/core/src/main/java/org/elasticsearch/index/analysis/NGramTokenizerFactory.java index 84da43497ac..424aa04e548 100644 --- a/core/src/main/java/org/elasticsearch/index/analysis/NGramTokenizerFactory.java +++ b/core/src/main/java/org/elasticsearch/index/analysis/NGramTokenizerFactory.java @@ -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 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); + } + }; } } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java b/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java index 3fb09679204..6e9c86b4c6a 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java @@ -604,14 +604,14 @@ public abstract class QueryBuilders { * Facilitates creating template query requests using an inline script */ public static TemplateQueryBuilder templateQuery(String template, Map 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 vars) { - return new TemplateQueryBuilder(template, templateType, vars); + return new TemplateQueryBuilder(new Template(template, templateType, null, null, vars)); } /** diff --git a/core/src/main/java/org/elasticsearch/index/store/Store.java b/core/src/main/java/org/elasticsearch/index/store/Store.java index c47770d9805..3b3074e2385 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -118,6 +118,7 @@ import static java.util.Collections.unmodifiableMap; * */ 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, Writeable { private static final ESLogger logger = Loggers.getLogger(MetadataSnapshot.class); - private static final Version FIRST_LUCENE_CHECKSUM_VERSION = Version.LUCENE_4_8; private final Map 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, 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 checksums, long lastVersion) throws IOException { // Make sure if clock goes backwards we still move version forwards: long nextVersion = Math.max(lastVersion+1, System.currentTimeMillis()); diff --git a/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltAnalyzers.java b/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltAnalyzers.java index 36795c66da4..6d5c3a8ed18 100644 --- a/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltAnalyzers.java +++ b/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltAnalyzers.java @@ -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; } diff --git a/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenFilters.java b/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenFilters.java index 631d8bac76d..027c75073f6 100644 --- a/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenFilters.java +++ b/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenFilters.java @@ -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); } }, diff --git a/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenizers.java b/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenizers.java index 7b1563388e0..c6a8255e4c3 100644 --- a/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenizers.java +++ b/core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenizers.java @@ -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); } }, From 4772b1fe9d57bf29dc93e08c3b05503b68f7c77a Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 8 Jan 2016 17:25:26 -0500 Subject: [PATCH 24/33] Remove tests for backwards compatibility we don't support any more --- .../analysis/NGramTokenizerFactoryTests.java | 77 ------------------- .../index/analysis/PreBuiltAnalyzerTests.java | 76 ------------------ 2 files changed, 153 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java index 6da1a7721a4..0cb3abbd1ff 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java @@ -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++) { diff --git a/core/src/test/java/org/elasticsearch/index/analysis/PreBuiltAnalyzerTests.java b/core/src/test/java/org/elasticsearch/index/analysis/PreBuiltAnalyzerTests.java index 297cab86f5e..1a88fcbc0c1 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/PreBuiltAnalyzerTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/PreBuiltAnalyzerTests.java @@ -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 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 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))); From db832cce068a6595c9e00b34c0ae751e653d0441 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Mon, 11 Jan 2016 04:52:26 -0500 Subject: [PATCH 25/33] improve logging messages a bit --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 2 +- .../indices/memory/IndexingMemoryController.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index cdc83fa8ffc..2d186ade638 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -543,7 +543,7 @@ public class InternalEngine extends Engine { 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 flush to write indexing buffer (heap size=[{}]) since version map is small (heap size=[{}])", + 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(); } diff --git a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 56166134df5..bce4f61f800 100644 --- a/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -301,8 +301,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent indexingBuffer.bytes()) { // OK we are now over-budget; fill the priority queue and ask largest shard(s) to refresh: - logger.debug("now write some indexing buffers: total indexing heap bytes used [{}] vs {} [{}], currently writing bytes [{}]", - new ByteSizeValue(totalBytesUsed), INDEX_BUFFER_SIZE_SETTING, indexingBuffer, new ByteSizeValue(totalBytesWriting)); PriorityQueue queue = new PriorityQueue<>(); for (IndexShard shard : availableShards()) { @@ -333,6 +331,9 @@ public class IndexingMemoryController extends AbstractLifecycleComponent 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)); From 5e7144f5c559026a4b36d3cdbca397a12cc914a3 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Mon, 11 Jan 2016 06:07:19 -0500 Subject: [PATCH 26/33] IMC is now just another IndexingOperationListener --- .../org/elasticsearch/index/IndexModule.java | 6 +++-- .../org/elasticsearch/index/IndexService.java | 8 ++++-- .../index/NodeServicesProvider.java | 9 +------ .../elasticsearch/index/shard/IndexShard.java | 15 +++-------- .../indices/IndexingMemoryController.java | 26 ++++++++++++++++--- .../elasticsearch/indices/IndicesService.java | 3 +-- .../elasticsearch/index/IndexModuleTests.java | 23 ++++++++-------- .../index/engine/InternalEngineTests.java | 2 +- 8 files changed, 50 insertions(+), 42 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index 68e7b2672fb..5189e4facb4 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -34,6 +34,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.cache.query.IndicesQueryCache; import org.elasticsearch.indices.mapper.MapperRegistry; @@ -241,7 +242,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, + IndexingMemoryController indexingMemoryController) throws IOException { final IndexSettings settings = indexSettings.newWithListener(settingsConsumers); IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get(); IndexEventListener eventListener = freeze(); @@ -263,6 +265,6 @@ public final class IndexModule { final BiFunction 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, indexingMemoryController); } } diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 3d1a9f8ed76..8963ba365ab 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -60,6 +60,7 @@ import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.AliasFilterParsingException; +import org.elasticsearch.indices.IndexingMemoryController; import org.elasticsearch.indices.InvalidAliasNameException; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.threadpool.ThreadPool; @@ -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 IndexingMemoryController indexingMemoryController; 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, + IndexingMemoryController indexingMemoryController) throws IOException { super(indexSettings); this.indexSettings = indexSettings; this.analysisService = registry.build(indexSettings); @@ -132,6 +135,7 @@ 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()); + this.indexingMemoryController = indexingMemoryController; } public int numberOfShards() { @@ -296,7 +300,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, slowLog, indexingMemoryController); } eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); diff --git a/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java b/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java index 84e93b219e3..a6d464cbaad 100644 --- a/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java +++ b/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java @@ -28,7 +28,6 @@ import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; -import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.ScriptService; import org.elasticsearch.threadpool.ThreadPool; @@ -50,10 +49,9 @@ public final class NodeServicesProvider { private final ScriptService scriptService; private final IndicesFieldDataCache indicesFieldDataCache; private final CircuitBreakerService circuitBreakerService; - private final IndexingMemoryController indexingMemoryController; @Inject - public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, TermVectorsService termVectorsService, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, IndicesFieldDataCache indicesFieldDataCache, CircuitBreakerService circuitBreakerService, IndexingMemoryController indexingMemoryController) { + public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, TermVectorsService termVectorsService, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, IndicesFieldDataCache indicesFieldDataCache, CircuitBreakerService circuitBreakerService) { this.threadPool = threadPool; this.indicesQueryCache = indicesQueryCache; this.termVectorsService = termVectorsService; @@ -64,7 +62,6 @@ public final class NodeServicesProvider { this.scriptService = scriptService; this.indicesFieldDataCache = indicesFieldDataCache; this.circuitBreakerService = circuitBreakerService; - this.indexingMemoryController = indexingMemoryController; } public ThreadPool getThreadPool() { @@ -104,8 +101,4 @@ public final class NodeServicesProvider { public CircuitBreakerService getCircuitBreakerService() { return circuitBreakerService; } - - public IndexingMemoryController getIndexingMemoryController() { - return indexingMemoryController; - } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index ef34ec015e3..d16d1c537d2 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -167,13 +167,12 @@ public class IndexShard extends AbstractIndexShardComponent { private final IndexEventListener indexEventListener; private final IndexSettings idxSettings; private final NodeServicesProvider provider; -<<<<<<< HEAD - /** How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh */ + /** 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(); -======= ->>>>>>> master private TimeValue refreshInterval; private volatile ScheduledFuture refreshScheduledFuture; @@ -216,7 +215,6 @@ public class IndexShard extends AbstractIndexShardComponent { * IndexingMemoryController}). */ private final AtomicBoolean active = new AtomicBoolean(); - private final IndexingMemoryController indexingMemoryController; public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService, @@ -273,7 +271,6 @@ public class IndexShard extends AbstractIndexShardComponent { this.engineConfig = newEngineConfig(translogConfig, cachingPolicy); this.flushThresholdSize = settings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB)); this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId); - this.indexingMemoryController = provider.getIndexingMemoryController(); this.provider = provider; this.searcherWrapper = indexSearcherWrapper; this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, newQueryShardContext()); @@ -510,9 +507,6 @@ public class IndexShard extends AbstractIndexShardComponent { throw ex; } - // Notify IMC so that it can go and check heap used by all indexing buffers periodically: - indexingMemoryController.bytesWritten(index.getTranslogLocation().size); - indexingOperationListeners.postIndex(index); return created; @@ -556,9 +550,6 @@ public class IndexShard extends AbstractIndexShardComponent { throw ex; } - // Notify IMC so that it can go and check heap used by all indexing buffers periodically: - indexingMemoryController.bytesWritten(delete.getTranslogLocation().size); - indexingOperationListeners.postDelete(delete); } diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index 08685cd76f0..e377ca91898 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -26,10 +26,12 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; 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.IndexShard; import org.elasticsearch.index.shard.IndexShardState; +import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.threadpool.ThreadPool; @@ -46,7 +48,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; -public class IndexingMemoryController extends AbstractComponent implements 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"; @@ -124,11 +126,19 @@ public class IndexingMemoryController extends AbstractComponent implements Close 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) { // it's fine to run it on the scheduler thread, no busy work - return threadPool.scheduleWithFixedDelay(statusChecker, interval); + if (threadPool != null) { + return threadPool.scheduleWithFixedDelay(statusChecker, interval); + } else { + // tests pass null for threadPool --> no periodic checking + return null; + } } @Override @@ -198,7 +208,17 @@ public class IndexingMemoryController extends AbstractComponent implements Close shard.deactivateThrottling(); } - static final class ShardAndBytesUsed implements Comparable { + @Override + 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 { final long bytesUsed; final IndexShard shard; diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index d751b242fcb..c3c0ffe4732 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -293,14 +293,13 @@ public class IndicesService extends AbstractLifecycleComponent 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; diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 50ac44bb202..a7b50198044 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -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; @@ -59,7 +60,6 @@ import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener; import org.elasticsearch.indices.mapper.MapperRegistry; -import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineService; @@ -109,8 +109,7 @@ public class IndexModuleTests extends ESTestCase { scriptEngines.addAll(Arrays.asList(scriptEngineServices)); ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), new ScriptContextRegistry(Collections.emptyList())); IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, Collections.emptySet(), new NamedWriteableRegistry()); - IndexingMemoryController indexingMemoryController = new IndexingMemoryController(settings, threadPool, null); - return new NodeServicesProvider(threadPool, indicesQueryCache, null, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService, indexingMemoryController); + return new NodeServicesProvider(threadPool, indicesQueryCache, null, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService); } @Override @@ -139,7 +138,7 @@ public class IndexModuleTests extends ESTestCase { IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); module.setSearcherWrapper((s) -> new Wrapper()); module.engineFactory.set(new MockEngineFactory(AssertingDirectoryReader.class)); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); assertTrue(indexService.getSearcherWrapper() instanceof Wrapper); assertSame(indexService.getEngineFactory(), module.engineFactory.get()); indexService.close("simon says", false); @@ -152,7 +151,7 @@ public class IndexModuleTests extends ESTestCase { IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); module.addIndexStore("foo_store", FooStore::new); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); assertTrue(indexService.getIndexStore() instanceof FooStore); try { module.addIndexStore("foo_store", FooStore::new); @@ -176,7 +175,7 @@ public class IndexModuleTests extends ESTestCase { Consumer listener = (s) -> {}; module.addIndexSettingsListener(listener); module.addIndexEventListener(eventListener); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); IndexSettings x = indexService.getIndexSettings(); assertEquals(x.getSettings().getAsMap(), indexSettings.getSettings().getAsMap()); assertEquals(x.getIndex(), index); @@ -206,7 +205,7 @@ public class IndexModuleTests extends ESTestCase { } catch (IllegalArgumentException ex) { } - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); IndexSettings x = indexService.getIndexSettings(); assertEquals(1, x.getUpdateListeners().size()); assertSame(x.getUpdateListeners().get(0), listener); @@ -233,7 +232,7 @@ public class IndexModuleTests extends ESTestCase { } }); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); SimilarityService similarityService = indexService.similarityService(); assertNotNull(similarityService.getSimilarity("my_similarity")); assertTrue(similarityService.getSimilarity("my_similarity").get() instanceof TestSimilarity); @@ -250,7 +249,7 @@ public class IndexModuleTests extends ESTestCase { .build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment)); try { - module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); } catch (IllegalArgumentException ex) { assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage()); } @@ -264,7 +263,7 @@ public class IndexModuleTests extends ESTestCase { .build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment)); try { - module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); } catch (IllegalArgumentException ex) { assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage()); } @@ -311,7 +310,7 @@ public class IndexModuleTests extends ESTestCase { assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]"); } - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); assertTrue(indexService.cache().query() instanceof CustomQueryCache); indexService.close("simon says", false); } @@ -321,7 +320,7 @@ public class IndexModuleTests extends ESTestCase { .put("path.home", createTempDir().toString()) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment)); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); assertTrue(indexService.cache().query() instanceof IndexQueryCache); indexService.close("simon says", false); } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index ab66ce57f78..24b8cdad2b4 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -95,7 +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.memory.IndexingMemoryController; +import org.elasticsearch.indices.IndexingMemoryController; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.test.DummyShardLock; From 5bd458282457cd52fb43f56c6df132175dba1483 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Mon, 11 Jan 2016 14:44:28 +0100 Subject: [PATCH 27/33] Release script: Ensure push to the right region with latest rpm-s3 --- dev-tools/prepare_release_candidate.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev-tools/prepare_release_candidate.py b/dev-tools/prepare_release_candidate.py index 31b07043389..e0baa754d0c 100644 --- a/dev-tools/prepare_release_candidate.py +++ b/dev-tools/prepare_release_candidate.py @@ -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) From d8af49eb911102c2583e768983d3845bcf225b68 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 11 Jan 2016 10:05:48 -0500 Subject: [PATCH 28/33] Revert "Fix blended terms for non-strings" This reverts commit 6bb01984b6cceab38d50dcb92ba8da5c6a5f28ab. It caused test failures. --- .../index/mapper/MappedFieldType.java | 2 +- .../index/search/MatchQuery.java | 27 ++----- .../index/search/MultiMatchQuery.java | 48 +----------- .../search/query/MultiMatchQueryIT.java | 76 ------------------- 4 files changed, 11 insertions(+), 142 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java index 28590c56705..5f8049b55fb 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java @@ -390,7 +390,7 @@ public abstract class MappedFieldType extends FieldType { } /** Creates a term associated with the field of this mapper for the given value */ - public Term createTerm(Object value) { + protected Term createTerm(Object value) { return new Term(name(), indexedValueForSearch(value)); } diff --git a/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java b/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java index 46dc9d3486a..1b213645ae5 100644 --- a/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java +++ b/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java @@ -212,6 +212,10 @@ public class MatchQuery { this.zeroTermsQuery = zeroTermsQuery; } + protected boolean forceAnalyzeQueryString() { + return false; + } + protected Analyzer getAnalyzer(MappedFieldType fieldType) { if (this.analyzer == null) { if (fieldType != null) { @@ -236,18 +240,9 @@ public class MatchQuery { field = fieldName; } - /* - * If the user forced an analyzer we really don't care if they are - * searching a type that wants term queries to be used with query string - * because the QueryBuilder will take care of it. If they haven't forced - * an analyzer then types like NumberFieldType that want terms with - * query string will blow up because their analyzer isn't capable of - * passing through QueryBuilder. - */ - boolean noForcedAnalyzer = this.analyzer == null; - if (fieldType != null && fieldType.useTermQueryWithQueryString() && noForcedAnalyzer) { + if (fieldType != null && fieldType.useTermQueryWithQueryString() && !forceAnalyzeQueryString()) { try { - return termQuery(fieldType, value); + return fieldType.termQuery(value, context); } catch (RuntimeException e) { if (lenient) { return null; @@ -256,7 +251,6 @@ public class MatchQuery { } } - Analyzer analyzer = getAnalyzer(fieldType); assert analyzer != null; MatchQueryBuilder builder = new MatchQueryBuilder(analyzer, fieldType); @@ -288,15 +282,6 @@ public class MatchQuery { } } - /** - * Creates a TermQuery-like-query for MappedFieldTypes that don't support - * QueryBuilder which is very string-ish. Just delegates to the - * MappedFieldType for MatchQuery but gets more complex for blended queries. - */ - protected Query termQuery(MappedFieldType fieldType, Object value) { - return fieldType.termQuery(value, context); - } - protected Query zeroTermsQuery() { return zeroTermsQuery == DEFAULT_ZERO_TERMS_QUERY ? Queries.newMatchNoDocsQuery() : Queries.newMatchAllQuery(); } diff --git a/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java b/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java index e6753538939..cf30c3dbe47 100644 --- a/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java +++ b/core/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java @@ -149,10 +149,6 @@ public class MultiMatchQuery extends MatchQuery { public boolean forceAnalyzeQueryString() { return false; } - - public Query termQuery(MappedFieldType fieldType, Object value) { - return fieldType.termQuery(value, context); - } } public class CrossFieldsQueryBuilder extends QueryBuilder { @@ -200,13 +196,8 @@ public class MultiMatchQuery extends MatchQuery { } else { blendedFields = null; } - /* - * We have to pick some field to pass through the superclass so - * we just pick the first field. It shouldn't matter because - * fields are already grouped by their analyzers/types. - */ - String representativeField = group.get(0).field; - Query q = parseGroup(type.matchQueryType(), representativeField, 1f, value, minimumShouldMatch); + final FieldAndFieldType fieldAndFieldType = group.get(0); + Query q = parseGroup(type.matchQueryType(), fieldAndFieldType.field, 1f, value, minimumShouldMatch); if (q != null) { queries.add(q); } @@ -215,28 +206,6 @@ public class MultiMatchQuery extends MatchQuery { return queries.isEmpty() ? null : queries; } - /** - * Pick the field for parsing. If any of the fields in the group do - * *not* useTermQueryWithQueryString then we return that one to force - * analysis. If some of the fields would useTermQueryWithQueryString - * then we assume that that parsing field's parser is good enough for - * them and return it. Otherwise we just return the first field. You - * should only get mixed groups like this when you force a certain - * analyzer on a query and use string and integer fields because of the - * way that grouping is done. That means that the use *asked* for the - * integer fields to be searched using a string analyzer so this is - * technically doing exactly what they asked for even if it is a bit - * funky. - */ - private String fieldForParsing(List group) { - for (FieldAndFieldType field: group) { - if (field.fieldType.useTermQueryWithQueryString()) { - return field.field; - } - } - return group.get(0).field; - } - @Override public boolean forceAnalyzeQueryString() { return blendedFields != null; @@ -262,11 +231,6 @@ public class MultiMatchQuery extends MatchQuery { } return BlendedTermQuery.dismaxBlendedQuery(terms, blendedBoost, tieBreaker); } - - @Override - public Query termQuery(MappedFieldType fieldType, Object value) { - return blendTerm(fieldType.createTerm(value), fieldType); - } } @Override @@ -302,11 +266,7 @@ public class MultiMatchQuery extends MatchQuery { } @Override - protected Query termQuery(MappedFieldType fieldType, Object value) { - if (queryBuilder == null) { - // Can be null when the MultiMatchQuery collapses into a MatchQuery - return super.termQuery(fieldType, value); - } - return queryBuilder.termQuery(fieldType, value); + protected boolean forceAnalyzeQueryString() { + return this.queryBuilder == null ? super.forceAnalyzeQueryString() : this.queryBuilder.forceAnalyzeQueryString(); } } diff --git a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java index 1e070f325b3..a789497235e 100644 --- a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.query; import com.carrotsearch.randomizedtesting.generators.RandomPicks; - import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; @@ -460,23 +459,6 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertHitCount(searchResponse, 1l); assertFirstHit(searchResponse, hasId("theone")); - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("captain america 15", "full_name", "first_name", "last_name", "category", "skill", "int-field") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .analyzer("category") - .operator(Operator.AND))).get(); - assertHitCount(searchResponse, 1l); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("captain america 15", "skill", "full_name", "first_name", "last_name", "category", "int-field") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .analyzer("category") - .operator(Operator.AND))).get(); - assertHitCount(searchResponse, 1l); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("captain america 15", "first_name", "last_name", "skill") .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) @@ -489,24 +471,6 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .analyzer("category"))).get(); assertFirstHit(searchResponse, hasId("theone")); - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("25 15", "first_name", "int-field", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .analyzer("category"))).get(); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("25 15", "int-field", "skill", "first_name") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .analyzer("category"))).get(); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("25 15", "int-field", "first_name", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .analyzer("category"))).get(); - assertFirstHit(searchResponse, hasId("theone")); - searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("captain america marvel hero", "first_name", "last_name", "category") .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) @@ -565,46 +529,6 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertFirstHit(searchResponse, hasId("ultimate2")); assertSecondHit(searchResponse, hasId("ultimate1")); assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); - - // Test group based on numeric fields - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("15", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("15", "skill", "first_name") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); - assertFirstHit(searchResponse, hasId("theone")); - - // Two numeric fields together caused trouble at one point! - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("15", "int-field", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("15", "int-field", "first_name", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS))).get(); - assertFirstHit(searchResponse, hasId("theone")); - - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("alpha 15", "first_name", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .lenient(true))).get(); - assertFirstHit(searchResponse, hasId("ultimate1")); - /* - * Doesn't find theone because "alpha 15" isn't a number and we don't - * break on spaces. - */ - assertHitCount(searchResponse, 1); - - // Lenient wasn't always properly lenient with two numeric fields - searchResponse = client().prepareSearch("test") - .setQuery(randomizeType(multiMatchQuery("alpha 15", "int-field", "first_name", "skill") - .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS) - .lenient(true))).get(); - assertFirstHit(searchResponse, hasId("ultimate1")); } private static final void assertEquivalent(String query, SearchResponse left, SearchResponse right) { From f782c8ac07b89679e0a2b2329a31836b0b7da9b1 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Mon, 11 Jan 2016 20:00:09 +0100 Subject: [PATCH 29/33] Merge pull request #15891 from sromocki/patch-1 Update crud.asciidoc, nouns mixed up --- docs/reference/migration/migrate_2_0/crud.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/migration/migrate_2_0/crud.asciidoc b/docs/reference/migration/migrate_2_0/crud.asciidoc index f79306a2bbd..ef3ba93e67e 100644 --- a/docs/reference/migration/migrate_2_0/crud.asciidoc +++ b/docs/reference/migration/migrate_2_0/crud.asciidoc @@ -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. From 07e83703227c0a73c4e90a6a5e8146e4e5e26df7 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Mon, 11 Jan 2016 17:16:28 -0500 Subject: [PATCH 30/33] feedback --- .../org/elasticsearch/index/IndexModule.java | 5 +-- .../org/elasticsearch/index/IndexService.java | 36 ++++++++++--------- .../elasticsearch/index/shard/IndexShard.java | 10 ++---- .../indices/IndexingMemoryController.java | 23 ++++++------ .../elasticsearch/index/IndexModuleTests.java | 18 +++++----- .../IndexingMemoryControllerTests.java | 4 +++ docs/reference/migration/migrate_3_0.asciidoc | 6 +++- 7 files changed, 57 insertions(+), 45 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index 5189e4facb4..9a8357daa26 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -29,6 +29,7 @@ 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; @@ -243,7 +244,7 @@ public final class IndexModule { } public IndexService newIndexService(NodeEnvironment environment, IndexService.ShardStoreDeleter shardStoreDeleter, NodeServicesProvider servicesProvider, MapperRegistry mapperRegistry, - IndexingMemoryController indexingMemoryController) throws IOException { + IndexingOperationListener... listeners) throws IOException { final IndexSettings settings = indexSettings.newWithListener(settingsConsumers); IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get(); IndexEventListener eventListener = freeze(); @@ -265,6 +266,6 @@ public final class IndexModule { final BiFunction 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, indexingMemoryController); + servicesProvider, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, listeners); } } diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 8963ba365ab..3c348ef8830 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -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; @@ -60,22 +72,10 @@ import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.AliasFilterParsingException; -import org.elasticsearch.indices.IndexingMemoryController; 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; @@ -103,7 +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 IndexingMemoryController indexingMemoryController; + private final IndexingOperationListener[] listeners; public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv, SimilarityService similarityService, @@ -116,7 +116,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC IndexEventListener eventListener, IndexModule.IndexSearcherWrapperFactory wrapperFactory, MapperRegistry mapperRegistry, - IndexingMemoryController indexingMemoryController) throws IOException { + IndexingOperationListener... listenersIn) throws IOException { super(indexSettings); this.indexSettings = indexSettings; this.analysisService = registry.build(indexSettings); @@ -135,7 +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()); - this.indexingMemoryController = indexingMemoryController; + + // 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() { @@ -300,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, indexingMemoryController); + 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"); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index d16d1c537d2..de9bb7e10f1 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -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 { @@ -532,7 +532,7 @@ public class IndexShard extends AbstractIndexShardComponent { public void delete(Engine.Delete delete) { ensureWriteAllowed(delete); - markLastWrite(); + active.set(true); delete = indexingOperationListeners.preDelete(delete); try { if (logger.isTraceEnabled()) { @@ -974,11 +974,6 @@ public class IndexShard extends AbstractIndexShardComponent { } } - /** Sets {@code active} to true if we were inactive. */ - private void markLastWrite() { - active.set(true); - } - private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardStateException { Engine.Operation.Origin origin = op.origin(); IndexShardState state = this.state; // one time volatile read @@ -1036,6 +1031,7 @@ public class IndexShard extends AbstractIndexShardComponent { } } + /** 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) { diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index e377ca91898..35bf4d3b62c 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -24,6 +24,7 @@ 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; @@ -133,12 +134,7 @@ public class IndexingMemoryController extends AbstractComponent implements Index protected ScheduledFuture scheduleTask(ThreadPool threadPool) { // it's fine to run it on the scheduler thread, no busy work - if (threadPool != null) { - return threadPool.scheduleWithFixedDelay(statusChecker, interval); - } else { - // tests pass null for threadPool --> no periodic checking - return null; - } + return threadPool.scheduleWithFixedDelay(statusChecker, interval); } @Override @@ -180,11 +176,16 @@ public class IndexingMemoryController extends AbstractComponent implements Index /** ask this shard to refresh, in the background, to free up heap */ protected void writeIndexingBufferAsync(IndexShard shard) { - threadPool.executor(ThreadPool.Names.REFRESH).execute(new Runnable() { + threadPool.executor(ThreadPool.Names.REFRESH).execute(new AbstractRunnable() { @Override - public void run() { + public void doRun() { shard.writeIndexingBuffer(); } + + @Override + public void onFailure(Throwable t) { + logger.warn("failed to write indexing buffer for shard [{}]; ignoring", t, shard.shardId()); + } }); } @@ -243,7 +244,7 @@ public class IndexingMemoryController extends AbstractComponent implements Index /** Shard calls this on each indexing/delete op */ public void bytesWritten(int bytes) { long totalBytes = bytesWrittenSinceCheck.addAndGet(bytes); - if (totalBytes > indexingBuffer.bytes()/30) { + while (totalBytes > indexingBuffer.bytes()/30) { if (runLock.tryLock()) { try { bytesWrittenSinceCheck.addAndGet(-totalBytes); @@ -251,10 +252,12 @@ public class IndexingMemoryController extends AbstractComponent implements Index // 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: - run(); + runUnlocked(); } finally { runLock.unlock(); } + } else { + break; } } } diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index a7b50198044..b83af9c0960 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -138,7 +138,7 @@ public class IndexModuleTests extends ESTestCase { IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); module.setSearcherWrapper((s) -> new Wrapper()); module.engineFactory.set(new MockEngineFactory(AssertingDirectoryReader.class)); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); assertTrue(indexService.getSearcherWrapper() instanceof Wrapper); assertSame(indexService.getEngineFactory(), module.engineFactory.get()); indexService.close("simon says", false); @@ -151,7 +151,7 @@ public class IndexModuleTests extends ESTestCase { IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); module.addIndexStore("foo_store", FooStore::new); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); assertTrue(indexService.getIndexStore() instanceof FooStore); try { module.addIndexStore("foo_store", FooStore::new); @@ -175,7 +175,7 @@ public class IndexModuleTests extends ESTestCase { Consumer listener = (s) -> {}; module.addIndexSettingsListener(listener); module.addIndexEventListener(eventListener); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); IndexSettings x = indexService.getIndexSettings(); assertEquals(x.getSettings().getAsMap(), indexSettings.getSettings().getAsMap()); assertEquals(x.getIndex(), index); @@ -205,7 +205,7 @@ public class IndexModuleTests extends ESTestCase { } catch (IllegalArgumentException ex) { } - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); IndexSettings x = indexService.getIndexSettings(); assertEquals(1, x.getUpdateListeners().size()); assertSame(x.getUpdateListeners().get(0), listener); @@ -232,7 +232,7 @@ public class IndexModuleTests extends ESTestCase { } }); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); SimilarityService similarityService = indexService.similarityService(); assertNotNull(similarityService.getSimilarity("my_similarity")); assertTrue(similarityService.getSimilarity("my_similarity").get() instanceof TestSimilarity); @@ -249,7 +249,7 @@ public class IndexModuleTests extends ESTestCase { .build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment)); try { - module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); } catch (IllegalArgumentException ex) { assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage()); } @@ -263,7 +263,7 @@ public class IndexModuleTests extends ESTestCase { .build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment)); try { - module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); } catch (IllegalArgumentException ex) { assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage()); } @@ -310,7 +310,7 @@ public class IndexModuleTests extends ESTestCase { assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]"); } - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); assertTrue(indexService.cache().query() instanceof CustomQueryCache); indexService.close("simon says", false); } @@ -320,7 +320,7 @@ public class IndexModuleTests extends ESTestCase { .put("path.home", createTempDir().toString()) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment)); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, null); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); assertTrue(indexService.cache().query() instanceof IndexQueryCache); indexService.close("simon says", false); } diff --git a/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java index f8f500237ad..afb9673508a 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java @@ -350,6 +350,10 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { shard.writeIndexingBuffer(); } + @Override + protected ScheduledFuture scheduleTask(ThreadPool threadPool) { + return null; + } }; for (int i = 0; i < 100; i++) { diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index d3c0b5a2d7a..681b50d5c2c 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -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 From 1de2081ed3674e38ca8dd432c8d51a295461f7c7 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 11 Jan 2016 17:26:00 -0500 Subject: [PATCH 31/33] Reintroduce five-minute and fifteen-minute load averages on Linux This commit reintroduces the five-minute and fifteen-minute load stats on Linux, and changes the format of the load_average field back to an array. --- .../org/elasticsearch/env/Environment.java | 8 ++-- .../org/elasticsearch/monitor/os/OsProbe.java | 38 ++++++++++++++++--- .../org/elasticsearch/monitor/os/OsStats.java | 26 ++++++++++--- .../rest/action/cat/RestNodesAction.java | 9 ++++- .../elasticsearch/bootstrap/security.policy | 3 ++ .../monitor/os/OsProbeTests.java | 27 +++++++++++-- docs/reference/cluster/nodes-stats.asciidoc | 3 +- docs/reference/migration/migrate_3_0.asciidoc | 33 +++++++++++----- .../test/cat.nodes/10_basic.yaml | 8 ++-- 9 files changed, 121 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/env/Environment.java b/core/src/main/java/org/elasticsearch/env/Environment.java index 7982c2f35ea..b6453a4707b 100644 --- a/core/src/main/java/org/elasticsearch/env/Environment.java +++ b/core/src/main/java/org/elasticsearch/env/Environment.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/monitor/os/OsProbe.java b/core/src/main/java/org/elasticsearch/monitor/os/OsProbe.java index 9e1ba39437a..49c6b9ddc5a 100644 --- a/core/src/main/java/org/elasticsearch/monitor/os/OsProbe.java +++ b/core/src/main/java/org/elasticsearch/monitor/os/OsProbe.java @@ -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 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); } diff --git a/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java b/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java index ebf7d9fafda..26f3afacb3f 100644 --- a/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java +++ b/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java @@ -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; } } diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index e86132a909e..07ff143e85d 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -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" : "-"); diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy index 151c91f5be2..ea525e55601 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -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"; }; diff --git a/core/src/test/java/org/elasticsearch/monitor/os/OsProbeTests.java b/core/src/test/java/org/elasticsearch/monitor/os/OsProbeTests.java index 5d482edafd1..4f4319e212c 100644 --- a/core/src/test/java/org/elasticsearch/monitor/os/OsProbeTests.java +++ b/core/src/test/java/org/elasticsearch/monitor/os/OsProbeTests.java @@ -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()); diff --git a/docs/reference/cluster/nodes-stats.asciidoc b/docs/reference/cluster/nodes-stats.asciidoc index cd6c7db2d41..144d6f72548 100644 --- a/docs/reference/cluster/nodes-stats.asciidoc +++ b/docs/reference/cluster/nodes-stats.asciidoc @@ -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 diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index d3c0b5a2d7a..2954c183403 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -552,17 +552,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. diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml index f41e14919f8..86f71c4bb1e 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml @@ -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: From 77e9eed6f18f22895e91161993358bbd84918d59 Mon Sep 17 00:00:00 2001 From: Felipe Forbeck Date: Wed, 16 Dec 2015 23:55:53 -0200 Subject: [PATCH 32/33] Docs explanation for unassigned shard reason codes. Fixes #14001 Closes #15912 --- docs/reference/cat/shards.asciidoc | 33 +++++++++++++++++++++++++----- 1 file changed, 28 insertions(+), 5 deletions(-) diff --git a/docs/reference/cat/shards.asciidoc b/docs/reference/cat/shards.asciidoc index a4359af258e..294b622bc8f 100644 --- a/docs/reference/cat/shards.asciidoc +++ b/docs/reference/cat/shards.asciidoc @@ -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 <> `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. From 9965c83ae42f570519465b0113d111735c0ce381 Mon Sep 17 00:00:00 2001 From: Felipe Forbeck Date: Sat, 19 Dec 2015 02:38:40 -0200 Subject: [PATCH 33/33] Documented how to define custom mappings for all indexes and all types Closes #15557 --- .../mapping/dynamic/templates.asciidoc | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/docs/reference/mapping/dynamic/templates.asciidoc b/docs/reference/mapping/dynamic/templates.asciidoc index b60c5f0510e..b903f1af066 100644 --- a/docs/reference/mapping/dynamic/templates.asciidoc +++ b/docs/reference/mapping/dynamic/templates.asciidoc @@ -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.