Upgrade to Lucene 4.8.1
This commit upgrades to the latest Lucene 4.8.1 release including the following bugfixes: * An IndexThrottle now kicks in when merges start falling behind limiting index threads to 1 until merges caught up. Closes #6066 * RateLimiter now kicks in at the configured rate where previously the limiter was limiting at ~8MB/sec almost all the time. Closes #6018
This commit is contained in:
parent
3731362ca8
commit
85a0b76dbb
|
@ -193,5 +193,25 @@ scheduler supports this setting:
|
|||
|
||||
`index.merge.scheduler.max_thread_count`::
|
||||
|
||||
The maximum number of threads to perform the merge operation. Defaults to
|
||||
`Math.max(1, Math.min(3, Runtime.getRuntime().availableProcessors() / 2))`.
|
||||
The maximum number of concurrent merge threads that may run at once. Defaults
|
||||
to `1` which works best with spinning-magnets disks. If you are using
|
||||
a good solid-state disk (SSD) instead then try setting this to `3`.
|
||||
|
||||
[float]
|
||||
==== SerialMergeScheduler
|
||||
|
||||
A merge scheduler that simply does each merge sequentially using the
|
||||
calling thread (blocking the operations that triggered the merge or the
|
||||
index operation). This merge scheduler has a merge thread pool that
|
||||
explicitly schedules merges, and it makes sure that merges are serial
|
||||
within a shard, yet concurrent across multiple shards.
|
||||
|
||||
The scheduler supports the following settings:
|
||||
|
||||
`index.merge.scheduler.max_merge_at_once`::
|
||||
|
||||
The maximum number of merges a single merge run performs. This setting prevents
|
||||
executing unlimited amount of merges in a loop until another shards has a
|
||||
chance to get a merge thread from the pool. If this limit is reached the
|
||||
merge thread returns to the pool and continues once the the call to a single
|
||||
shards is executed. The default is `5`
|
||||
|
|
6
pom.xml
6
pom.xml
|
@ -31,7 +31,7 @@
|
|||
</parent>
|
||||
|
||||
<properties>
|
||||
<lucene.version>4.8.0</lucene.version>
|
||||
<lucene.version>4.8.1</lucene.version>
|
||||
<tests.jvms>auto</tests.jvms>
|
||||
<tests.shuffle>true</tests.shuffle>
|
||||
<tests.output>onerror</tests.output>
|
||||
|
@ -47,6 +47,10 @@
|
|||
<id>Codehaus Snapshots</id>
|
||||
<url>http://repository.codehaus.org/</url>
|
||||
</repository>
|
||||
<repository>
|
||||
<id>Apache Maven Repository</id>
|
||||
<url>https://repository.apache.org/content/repositories/releases/</url>
|
||||
</repository>
|
||||
</repositories>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -81,6 +81,7 @@ public final class RateLimitedFSDirectory extends FilterDirectory{
|
|||
private final BufferedIndexOutput bufferedDelegate;
|
||||
private final RateLimiter rateLimiter;
|
||||
private final StoreRateLimiting.Listener rateListener;
|
||||
private long bytesSinceLastRateLimit;
|
||||
|
||||
RateLimitedIndexOutput(final RateLimiter rateLimiter, final StoreRateLimiting.Listener rateListener, final IndexOutput delegate) {
|
||||
super(delegate instanceof BufferedIndexOutput ? ((BufferedIndexOutput) delegate).getBufferSize() : BufferedIndexOutput.DEFAULT_BUFFER_SIZE);
|
||||
|
@ -97,7 +98,12 @@ public final class RateLimitedFSDirectory extends FilterDirectory{
|
|||
|
||||
@Override
|
||||
protected void flushBuffer(byte[] b, int offset, int len) throws IOException {
|
||||
rateListener.onPause(rateLimiter.pause(len));
|
||||
bytesSinceLastRateLimit += len;
|
||||
if (bytesSinceLastRateLimit >= rateLimiter.getMinPauseCheckBytes()) {
|
||||
long pause = rateLimiter.pause(bytesSinceLastRateLimit);
|
||||
bytesSinceLastRateLimit = 0;
|
||||
rateListener.onPause(pause);
|
||||
}
|
||||
if (bufferedDelegate != null) {
|
||||
bufferedDelegate.flushBuffer(b, offset, len);
|
||||
} else {
|
||||
|
|
|
@ -19,7 +19,19 @@
|
|||
|
||||
package org.elasticsearch.index.engine.internal;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
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 org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -28,6 +40,7 @@ import org.apache.lucene.search.SearcherFactory;
|
|||
import org.apache.lucene.search.SearcherManager;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
import org.apache.lucene.store.NoLockFactory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
|
@ -39,6 +52,8 @@ import org.elasticsearch.common.collect.MapBuilder;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.lucene.HashedBytesRef;
|
||||
import org.elasticsearch.common.lucene.LoggerInfoStream;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
|
@ -75,18 +90,7 @@ import org.elasticsearch.index.translog.TranslogStreams;
|
|||
import org.elasticsearch.indices.warmer.IndicesWarmer;
|
||||
import org.elasticsearch.indices.warmer.InternalIndicesWarmer;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -163,6 +167,8 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
|
||||
private SegmentInfos lastCommittedSegmentInfos;
|
||||
|
||||
private IndexThrottle throttle;
|
||||
|
||||
@Inject
|
||||
public InternalEngine(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool,
|
||||
IndexSettingsService indexSettingsService, ShardIndexingService indexingService, @Nullable IndicesWarmer warmer,
|
||||
|
@ -257,6 +263,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
}
|
||||
try {
|
||||
this.indexWriter = createWriter();
|
||||
mergeScheduler.removeListener(this.throttle);
|
||||
this.throttle = new IndexThrottle(mergeScheduler.getMaxMerges(), logger);
|
||||
mergeScheduler.addListener(throttle);
|
||||
} catch (IOException e) {
|
||||
throw new EngineCreationFailureException(shardId, "failed to create engine", e);
|
||||
}
|
||||
|
@ -373,7 +382,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
if (writer == null) {
|
||||
throw new EngineClosedException(shardId, failedEngine);
|
||||
}
|
||||
innerCreate(create, writer);
|
||||
try (Releasable r = throttle.acquireThrottle()) {
|
||||
innerCreate(create, writer);
|
||||
}
|
||||
dirty = true;
|
||||
possibleMergeNeeded = true;
|
||||
flushNeeded = true;
|
||||
|
@ -462,8 +473,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
if (writer == null) {
|
||||
throw new EngineClosedException(shardId, failedEngine);
|
||||
}
|
||||
|
||||
innerIndex(index, writer);
|
||||
try (Releasable r = throttle.acquireThrottle()) {
|
||||
innerIndex(index, writer);
|
||||
}
|
||||
dirty = true;
|
||||
possibleMergeNeeded = true;
|
||||
flushNeeded = true;
|
||||
|
@ -744,7 +756,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
// to be allocated to a different node
|
||||
currentIndexWriter().close(false);
|
||||
indexWriter = createWriter();
|
||||
|
||||
mergeScheduler.removeListener(this.throttle);
|
||||
this.throttle = new IndexThrottle(mergeScheduler.getMaxMerges(), this.logger);
|
||||
mergeScheduler.addListener(throttle);
|
||||
// commit on a just opened writer will commit even if there are no changes done to it
|
||||
// we rely on that for the commit data translog id key
|
||||
if (flushNeeded || flush.force()) {
|
||||
|
@ -1559,4 +1573,75 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
return aBoolean != null && aBoolean.booleanValue();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static final class IndexThrottle implements MergeSchedulerProvider.Listener {
|
||||
|
||||
private static final InternalLock NOOP_LOCK = new InternalLock(new NoOpLock());
|
||||
private final InternalLock lockReference = new InternalLock(new ReentrantLock());
|
||||
private final AtomicInteger numMergesInFlight = new AtomicInteger(0);
|
||||
private final AtomicBoolean isThrottling = new AtomicBoolean();
|
||||
private final int maxNumMerges;
|
||||
private final ESLogger logger;
|
||||
|
||||
private volatile InternalLock lock = NOOP_LOCK;
|
||||
|
||||
public IndexThrottle(int maxNumMerges, ESLogger logger) {
|
||||
this.maxNumMerges = maxNumMerges;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
public Releasable acquireThrottle() {
|
||||
return lock.acquire();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeMerge(OnGoingMerge merge) {
|
||||
if (numMergesInFlight.incrementAndGet() > maxNumMerges) {
|
||||
if (isThrottling.getAndSet(true) == false) {
|
||||
logger.info("now throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges);
|
||||
}
|
||||
lock = lockReference;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterMerge(OnGoingMerge merge) {
|
||||
if (numMergesInFlight.decrementAndGet() < maxNumMerges) {
|
||||
if (isThrottling.getAndSet(false)) {
|
||||
logger.info("stop throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges);
|
||||
}
|
||||
lock = NOOP_LOCK;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static final class NoOpLock implements Lock {
|
||||
|
||||
@Override
|
||||
public void lock() {}
|
||||
|
||||
@Override
|
||||
public void lockInterruptibly() throws InterruptedException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tryLock() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unlock() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Condition newCondition() {
|
||||
throw new UnsupportedOperationException("NoOpLock can't provide a condition");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -73,7 +73,11 @@ public class ConcurrentMergeSchedulerProvider extends MergeSchedulerProvider {
|
|||
@Override
|
||||
public MergeScheduler buildMergeScheduler() {
|
||||
CustomConcurrentMergeScheduler concurrentMergeScheduler = new CustomConcurrentMergeScheduler(logger, shardId, this);
|
||||
concurrentMergeScheduler.setMaxMergesAndThreads(maxMergeCount, maxThreadCount);
|
||||
// nocommit but this doesn't handle SMS ... should we even expose/allow SMS? or, if user requests SMS can we just use CMS(1,1),
|
||||
// which would then stall if there are 2 merges in flight, and unstall once we are back to 1 or 0 merges
|
||||
// NOTE: we pass maxMergeCount+1 here so that CMS will allow one too many merges to kick off which then allows
|
||||
// InternalEngine.IndexThrottle to detect too-many-merges and throttle:
|
||||
concurrentMergeScheduler.setMaxMergesAndThreads(maxMergeCount+1, maxThreadCount);
|
||||
schedulers.add(concurrentMergeScheduler);
|
||||
return concurrentMergeScheduler;
|
||||
}
|
||||
|
@ -101,6 +105,10 @@ public class ConcurrentMergeSchedulerProvider extends MergeSchedulerProvider {
|
|||
indexSettingsService.removeListener(applySettings);
|
||||
}
|
||||
|
||||
public int getMaxMerges() {
|
||||
return this.maxMergeCount;
|
||||
}
|
||||
|
||||
public static class CustomConcurrentMergeScheduler extends TrackingConcurrentMergeScheduler {
|
||||
|
||||
private final ShardId shardId;
|
||||
|
|
|
@ -122,6 +122,9 @@ public abstract class MergeSchedulerProvider extends AbstractIndexShardComponent
|
|||
return scheduler;
|
||||
}
|
||||
|
||||
/** Maximum number of allowed running merges before index throttling kicks in. */
|
||||
public abstract int getMaxMerges();
|
||||
|
||||
protected abstract MergeScheduler buildMergeScheduler();
|
||||
|
||||
public abstract MergeStats stats();
|
||||
|
|
|
@ -35,6 +35,8 @@ public class RateLimitingInputStream extends InputStream {
|
|||
|
||||
private final Listener listener;
|
||||
|
||||
private long bytesSinceLastRateLimit;
|
||||
|
||||
public interface Listener {
|
||||
void onPause(long nanos);
|
||||
}
|
||||
|
@ -45,13 +47,21 @@ public class RateLimitingInputStream extends InputStream {
|
|||
this.listener = listener;
|
||||
}
|
||||
|
||||
private void maybePause(int bytes) {
|
||||
bytesSinceLastRateLimit += bytes;
|
||||
if (bytesSinceLastRateLimit >= rateLimiter.getMinPauseCheckBytes()) {
|
||||
long pause = rateLimiter.pause(bytesSinceLastRateLimit);
|
||||
bytesSinceLastRateLimit = 0;
|
||||
if (pause > 0) {
|
||||
listener.onPause(pause);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
int b = delegate.read();
|
||||
long pause = rateLimiter.pause(1);
|
||||
if (pause > 0) {
|
||||
listener.onPause(pause);
|
||||
}
|
||||
maybePause(1);
|
||||
return b;
|
||||
}
|
||||
|
||||
|
@ -64,7 +74,7 @@ public class RateLimitingInputStream extends InputStream {
|
|||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
int n = delegate.read(b, off, len);
|
||||
if (n > 0) {
|
||||
listener.onPause(rateLimiter.pause(n));
|
||||
maybePause(n);
|
||||
}
|
||||
return n;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue