|
|
@@ -20,6 +20,7 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
|
|
import org.elasticsearch.core.Nullable;
|
|
|
import org.elasticsearch.core.Releasable;
|
|
|
import org.elasticsearch.core.TimeValue;
|
|
|
+import org.elasticsearch.core.Tuple;
|
|
|
import org.elasticsearch.env.NodeEnvironment;
|
|
|
import org.elasticsearch.index.engine.ThreadPoolMergeScheduler.MergeTask;
|
|
|
import org.elasticsearch.monitor.fs.FsInfo;
|
|
|
@@ -28,6 +29,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|
|
|
|
|
import java.io.Closeable;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Comparator;
|
|
|
import java.util.IdentityHashMap;
|
|
|
@@ -59,10 +61,7 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
/** How frequently we check disk usage (default: 5 seconds). */
|
|
|
public static final Setting<TimeValue> INDICES_MERGE_DISK_CHECK_INTERVAL_SETTING = Setting.positiveTimeSetting(
|
|
|
"indices.merge.disk.check_interval",
|
|
|
- // disabled by default
|
|
|
- // there's currently a problem where (aborting) merges are blocked when shards are closed (because disk space is insufficient)
|
|
|
- // see: https://github.com/elastic/elasticsearch/issues/129335
|
|
|
- TimeValue.timeValueSeconds(0),
|
|
|
+ TimeValue.timeValueSeconds(5),
|
|
|
Property.Dynamic,
|
|
|
Property.NodeScope
|
|
|
);
|
|
|
@@ -294,6 +293,10 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
return queuedMergeTasks.isQueueEmpty() && runningMergeTasks.isEmpty() && ioThrottledMergeTasksCount.get() == 0L;
|
|
|
}
|
|
|
|
|
|
+ public boolean isMergingBlockedDueToInsufficientDiskSpace() {
|
|
|
+ return availableDiskSpacePeriodicMonitor.isScheduled() && queuedMergeTasks.queueHeadIsOverTheAvailableBudget();
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Enqueues a runnable that executes exactly one merge task, the smallest that is runnable at some point in time.
|
|
|
* A merge task is not runnable if its scheduler already reached the configured max-allowed concurrency level.
|
|
|
@@ -550,9 +553,8 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
|
|
|
static class MergeTaskPriorityBlockingQueue extends PriorityBlockingQueueWithBudget<MergeTask> {
|
|
|
MergeTaskPriorityBlockingQueue() {
|
|
|
- // start with 0 budget (so takes on this queue will always block until {@link #updateBudget} is invoked)
|
|
|
- // use the estimated *remaining* merge size as the budget function so that the disk space budget of taken (in-use) elements is
|
|
|
- // updated according to the remaining disk space requirements of the currently running merge tasks
|
|
|
+ // by default, start with 0 budget (so takes on this queue will always block until the first {@link #updateBudget} is invoked)
|
|
|
+ // use the estimated *remaining* merge size as the budget function so that the disk space budget of elements is updated
|
|
|
super(MergeTask::estimatedRemainingMergeSize, 0L);
|
|
|
}
|
|
|
|
|
|
@@ -563,7 +565,7 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
|
|
|
// exposed for tests
|
|
|
MergeTask peekQueue() {
|
|
|
- return enqueuedByBudget.peek();
|
|
|
+ return enqueuedByBudget.peek().v1();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -573,7 +575,7 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
*/
|
|
|
static class PriorityBlockingQueueWithBudget<E> {
|
|
|
private final ToLongFunction<? super E> budgetFunction;
|
|
|
- protected final PriorityQueue<E> enqueuedByBudget;
|
|
|
+ protected final PriorityQueue<Tuple<E, Long>> enqueuedByBudget;
|
|
|
private final IdentityHashMap<ElementWithReleasableBudget, Long> unreleasedBudgetPerElement;
|
|
|
private final ReentrantLock lock;
|
|
|
private final Condition elementAvailable;
|
|
|
@@ -581,7 +583,7 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
|
|
|
PriorityBlockingQueueWithBudget(ToLongFunction<? super E> budgetFunction, long initialAvailableBudget) {
|
|
|
this.budgetFunction = budgetFunction;
|
|
|
- this.enqueuedByBudget = new PriorityQueue<>(64, Comparator.comparingLong(budgetFunction));
|
|
|
+ this.enqueuedByBudget = new PriorityQueue<>(64, Comparator.comparingLong(Tuple::v2));
|
|
|
this.unreleasedBudgetPerElement = new IdentityHashMap<>();
|
|
|
this.lock = new ReentrantLock();
|
|
|
this.elementAvailable = lock.newCondition();
|
|
|
@@ -592,7 +594,7 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
final ReentrantLock lock = this.lock;
|
|
|
lock.lock();
|
|
|
try {
|
|
|
- enqueuedByBudget.offer(e);
|
|
|
+ enqueuedByBudget.offer(new Tuple<>(e, budgetFunction.applyAsLong(e)));
|
|
|
elementAvailable.signal();
|
|
|
} finally {
|
|
|
lock.unlock();
|
|
|
@@ -608,14 +610,14 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
final ReentrantLock lock = this.lock;
|
|
|
lock.lockInterruptibly();
|
|
|
try {
|
|
|
- E peek;
|
|
|
- long peekBudget;
|
|
|
+ Tuple<E, Long> head;
|
|
|
// blocks until the smallest budget element fits the currently available budget
|
|
|
- while ((peek = enqueuedByBudget.peek()) == null || (peekBudget = budgetFunction.applyAsLong(peek)) > availableBudget) {
|
|
|
+ while ((head = enqueuedByBudget.peek()) == null || head.v2() > availableBudget) {
|
|
|
elementAvailable.await();
|
|
|
}
|
|
|
+ head = enqueuedByBudget.poll();
|
|
|
// deducts and holds up that element's budget from the available budget
|
|
|
- return newElementWithReleasableBudget(enqueuedByBudget.poll(), peekBudget);
|
|
|
+ return newElementWithReleasableBudget(head.v1(), head.v2());
|
|
|
} finally {
|
|
|
lock.unlock();
|
|
|
}
|
|
|
@@ -623,7 +625,7 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
|
|
|
/**
|
|
|
* Updates the available budged given the passed-in argument, from which it deducts the budget hold up by taken elements
|
|
|
- * that are still in use. The budget of in-use elements is also updated (by re-applying the budget function).
|
|
|
+ * that are still in use. The elements budget is also updated by re-applying the budget function.
|
|
|
* The newly updated budget is used to potentially block {@link #take()} operations if the smallest-budget enqueued element
|
|
|
* is over this newly computed available budget.
|
|
|
*/
|
|
|
@@ -632,9 +634,11 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
lock.lock();
|
|
|
try {
|
|
|
this.availableBudget = availableBudget;
|
|
|
- // update the per-element budget (these are all the elements that are using any budget)
|
|
|
+ // updates the budget of enqueued elements (and possibly reorders the priority queue)
|
|
|
+ updateBudgetOfEnqueuedElementsAndReorderQueue();
|
|
|
+ // update the budget of dequeued, but still in-use elements (these are the elements that are consuming budget)
|
|
|
unreleasedBudgetPerElement.replaceAll((e, v) -> budgetFunction.applyAsLong(e.element()));
|
|
|
- // available budget is decreased by the used per-element budget (for all dequeued elements that are still in use)
|
|
|
+ // the available budget is decreased by the budget of still in-use elements (dequeued elements that are still in-use)
|
|
|
this.availableBudget -= unreleasedBudgetPerElement.values().stream().mapToLong(i -> i).sum();
|
|
|
elementAvailable.signalAll();
|
|
|
} finally {
|
|
|
@@ -642,10 +646,38 @@ public class ThreadPoolMergeExecutorService implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void updateBudgetOfEnqueuedElementsAndReorderQueue() {
|
|
|
+ assert this.lock.isHeldByCurrentThread();
|
|
|
+ int queueSizeBefore = enqueuedByBudget.size();
|
|
|
+ var it = enqueuedByBudget.iterator();
|
|
|
+ List<Tuple<E, Long>> elementsToReorder = new ArrayList<>();
|
|
|
+ while (it.hasNext()) {
|
|
|
+ var elementWithBudget = it.next();
|
|
|
+ Long previousBudget = elementWithBudget.v2();
|
|
|
+ long latestBudget = budgetFunction.applyAsLong(elementWithBudget.v1());
|
|
|
+ if (previousBudget.equals(latestBudget) == false) {
|
|
|
+ // the budget (estimation) of an enqueued element has changed
|
|
|
+ // this element will be reordered by removing and reinserting using the latest budget (estimation)
|
|
|
+ it.remove();
|
|
|
+ elementsToReorder.add(new Tuple<>(elementWithBudget.v1(), latestBudget));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // reinsert elements based on the latest budget (estimation)
|
|
|
+ for (var reorderedElement : elementsToReorder) {
|
|
|
+ enqueuedByBudget.offer(reorderedElement);
|
|
|
+ }
|
|
|
+ assert queueSizeBefore == enqueuedByBudget.size();
|
|
|
+ }
|
|
|
+
|
|
|
boolean isQueueEmpty() {
|
|
|
return enqueuedByBudget.isEmpty();
|
|
|
}
|
|
|
|
|
|
+ boolean queueHeadIsOverTheAvailableBudget() {
|
|
|
+ var head = enqueuedByBudget.peek();
|
|
|
+ return head != null && head.v2() > availableBudget;
|
|
|
+ }
|
|
|
+
|
|
|
int queueSize() {
|
|
|
return enqueuedByBudget.size();
|
|
|
}
|