Skip to content

Commit

Permalink
Periodically try to reassign unassigned persistent tasks (#36069)
Browse files Browse the repository at this point in the history
Previously persistent task assignment was checked in the
following situations:

- Persistent tasks are changed
- A node joins or leaves the cluster
- The routing table is changed
- Custom metadata in the cluster state is changed
- A new master node is elected

However, there could be situations when a persistent
task that could not be assigned to a node could become
assignable due to some other change, such as memory
usage on the nodes.

This change adds a timed recheck of persistent task
assignment to account for such situations.  The timer
is suspended while checks triggered by cluster state
changes are in-flight to avoid adding burden to an
already busy cluster.

Closes #35792
  • Loading branch information
droberts195 committed Dec 13, 2018
1 parent 34d7cc1 commit 13cb0fb
Show file tree
Hide file tree
Showing 12 changed files with 711 additions and 120 deletions.
14 changes: 12 additions & 2 deletions docs/reference/modules/cluster/misc.asciidoc
Expand Up @@ -135,10 +135,10 @@ Plugins can create a kind of tasks called persistent tasks. Those tasks are
usually long-live tasks and are stored in the cluster state, allowing the
tasks to be revived after a full cluster restart.

Every time a persistent task is created, the master nodes takes care of
Every time a persistent task is created, the master node takes care of
assigning the task to a node of the cluster, and the assigned node will then
pick up the task and execute it locally. The process of assigning persistent
tasks to nodes is controlled by the following property, which can be updated
tasks to nodes is controlled by the following properties, which can be updated
dynamically:

`cluster.persistent_tasks.allocation.enable`::
Expand All @@ -153,3 +153,13 @@ This setting does not affect the persistent tasks that are already being execute
Only newly created persistent tasks, or tasks that must be reassigned (after a node
left the cluster, for example), are impacted by this setting.
--

`cluster.persistent_tasks.allocation.recheck_interval`::

The master node will automatically check whether persistent tasks need to
be assigned when the cluster state changes significantly. However, there
may be other factors, such as memory usage, that affect whether persistent
tasks can be assigned to nodes but do not cause the cluster state to change.
This setting controls how often assignment checks are performed to react to
these factors. The default is 30 seconds. The minimum permitted value is 10
seconds.
Expand Up @@ -92,6 +92,7 @@
import org.elasticsearch.monitor.os.OsService;
import org.elasticsearch.monitor.process.ProcessService;
import org.elasticsearch.node.Node;
import org.elasticsearch.persistent.PersistentTasksClusterService;
import org.elasticsearch.persistent.decider.EnableAssignmentDecider;
import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.repositories.fs.FsRepository;
Expand Down Expand Up @@ -456,6 +457,7 @@ public void apply(Settings value, Settings current, Settings previous) {
Node.BREAKER_TYPE_KEY,
OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING,
IndexGraveyard.SETTING_MAX_TOMBSTONES,
PersistentTasksClusterService.CLUSTER_TASKS_ALLOCATION_RECHECK_INTERVAL_SETTING,
EnableAssignmentDecider.CLUSTER_TASKS_ALLOCATION_ENABLE_SETTING,
PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING,
PeerFinder.DISCOVERY_REQUEST_PEERS_TIMEOUT_SETTING,
Expand Down
@@ -0,0 +1,184 @@
/*
* 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.common.util.concurrent;

import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.threadpool.ThreadPool;

import java.io.Closeable;
import java.util.Objects;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;

/**
* A base class for tasks that need to repeat.
*/
public abstract class AbstractAsyncTask implements Runnable, Closeable {

private final Logger logger;
private final ThreadPool threadPool;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final boolean autoReschedule;
private volatile ScheduledFuture<?> scheduledFuture;
private volatile boolean isScheduledOrRunning;
private volatile Exception lastThrownException;
private volatile TimeValue interval;

protected AbstractAsyncTask(Logger logger, ThreadPool threadPool, TimeValue interval, boolean autoReschedule) {
this.logger = logger;
this.threadPool = threadPool;
this.interval = interval;
this.autoReschedule = autoReschedule;
}

/**
* Change the interval between runs.
* If a future run is scheduled then this will reschedule it.
* @param interval The new interval between runs.
*/
public synchronized void setInterval(TimeValue interval) {
this.interval = interval;
if (scheduledFuture != null) {
rescheduleIfNecessary();
}
}

public TimeValue getInterval() {
return interval;
}

/**
* Test any external conditions that determine whether the task
* should be scheduled. This method does *not* need to test if
* the task is closed, as being closed automatically prevents
* scheduling.
* @return Should the task be scheduled to run?
*/
protected abstract boolean mustReschedule();

/**
* Schedule the task to run after the configured interval if it
* is not closed and any further conditions imposed by derived
* classes are met. Any previously scheduled invocation is
* cancelled.
*/
public synchronized void rescheduleIfNecessary() {
if (isClosed()) {
return;
}
if (scheduledFuture != null) {
FutureUtils.cancel(scheduledFuture);
}
if (interval.millis() > 0 && mustReschedule()) {
if (logger.isTraceEnabled()) {
logger.trace("scheduling {} every {}", toString(), interval);
}
scheduledFuture = threadPool.schedule(interval, getThreadPool(), this);
isScheduledOrRunning = true;
} else {
logger.trace("scheduled {} disabled", toString());
scheduledFuture = null;
isScheduledOrRunning = false;
}
}

public boolean isScheduled() {
// Currently running counts as scheduled to avoid an oscillating return value
// from this method when a task is repeatedly running and rescheduling itself.
return isScheduledOrRunning;
}

/**
* Cancel any scheduled run, but do not prevent subsequent restarts.
*/
public synchronized void cancel() {
FutureUtils.cancel(scheduledFuture);
scheduledFuture = null;
isScheduledOrRunning = false;
}

/**
* Cancel any scheduled run
*/
@Override
public synchronized void close() {
if (closed.compareAndSet(false, true)) {
cancel();
}
}

public boolean isClosed() {
return this.closed.get();
}

@Override
public final void run() {
synchronized (this) {
scheduledFuture = null;
isScheduledOrRunning = autoReschedule;
}
try {
runInternal();
} catch (Exception ex) {
if (lastThrownException == null || sameException(lastThrownException, ex) == false) {
// prevent the annoying fact of logging the same stuff all the time with an interval of 1 sec will spam all your logs
logger.warn(
() -> new ParameterizedMessage(
"failed to run task {} - suppressing re-occurring exceptions unless the exception changes",
toString()),
ex);
lastThrownException = ex;
}
} finally {
if (autoReschedule) {
rescheduleIfNecessary();
}
}
}

private static boolean sameException(Exception left, Exception right) {
if (left.getClass() == right.getClass()) {
if (Objects.equals(left.getMessage(), right.getMessage())) {
StackTraceElement[] stackTraceLeft = left.getStackTrace();
StackTraceElement[] stackTraceRight = right.getStackTrace();
if (stackTraceLeft.length == stackTraceRight.length) {
for (int i = 0; i < stackTraceLeft.length; i++) {
if (stackTraceLeft[i].equals(stackTraceRight[i]) == false) {
return false;
}
}
return true;
}
}
}
return false;
}

protected abstract void runInternal();

/**
* Use the same threadpool by default.
* Derived classes can change this if required.
*/
protected String getThreadPool() {
return ThreadPool.Names.SAME;
}
}
95 changes: 6 additions & 89 deletions server/src/main/java/org/elasticsearch/index/IndexService.java
Expand Up @@ -38,8 +38,8 @@
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.AbstractAsyncTask;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.env.NodeEnvironment;
Expand Down Expand Up @@ -87,7 +87,6 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
Expand Down Expand Up @@ -805,100 +804,18 @@ private void maybeSyncGlobalCheckpoints() {
}
}

abstract static class BaseAsyncTask implements Runnable, Closeable {
abstract static class BaseAsyncTask extends AbstractAsyncTask {
protected final IndexService indexService;
protected final ThreadPool threadPool;
private final TimeValue interval;
private ScheduledFuture<?> scheduledFuture;
private final AtomicBoolean closed = new AtomicBoolean(false);
private volatile Exception lastThrownException;

BaseAsyncTask(IndexService indexService, TimeValue interval) {
super(indexService.logger, indexService.threadPool, interval, true);
this.indexService = indexService;
this.threadPool = indexService.getThreadPool();
this.interval = interval;
onTaskCompletion();
rescheduleIfNecessary();
}

boolean mustReschedule() {
protected boolean mustReschedule() {
// don't re-schedule if its closed or if we don't have a single shard here..., we are done
return indexService.closed.get() == false
&& closed.get() == false && interval.millis() > 0;
}

private synchronized void onTaskCompletion() {
if (mustReschedule()) {
if (indexService.logger.isTraceEnabled()) {
indexService.logger.trace("scheduling {} every {}", toString(), interval);
}
this.scheduledFuture = threadPool.schedule(interval, getThreadPool(), BaseAsyncTask.this);
} else {
indexService.logger.trace("scheduled {} disabled", toString());
this.scheduledFuture = null;
}
}

boolean isScheduled() {
return scheduledFuture != null;
}

@Override
public final void run() {
try {
runInternal();
} catch (Exception ex) {
if (lastThrownException == null || sameException(lastThrownException, ex) == false) {
// prevent the annoying fact of logging the same stuff all the time with an interval of 1 sec will spam all your logs
indexService.logger.warn(
() -> new ParameterizedMessage(
"failed to run task {} - suppressing re-occurring exceptions unless the exception changes",
toString()),
ex);
lastThrownException = ex;
}
} finally {
onTaskCompletion();
}
}

private static boolean sameException(Exception left, Exception right) {
if (left.getClass() == right.getClass()) {
if (Objects.equals(left.getMessage(), right.getMessage())) {
StackTraceElement[] stackTraceLeft = left.getStackTrace();
StackTraceElement[] stackTraceRight = right.getStackTrace();
if (stackTraceLeft.length == stackTraceRight.length) {
for (int i = 0; i < stackTraceLeft.length; i++) {
if (stackTraceLeft[i].equals(stackTraceRight[i]) == false) {
return false;
}
}
return true;
}
}
}
return false;
}

protected abstract void runInternal();

protected String getThreadPool() {
return ThreadPool.Names.SAME;
}

@Override
public synchronized void close() {
if (closed.compareAndSet(false, true)) {
FutureUtils.cancel(scheduledFuture);
scheduledFuture = null;
}
}

TimeValue getInterval() {
return interval;
}

boolean isClosed() {
return this.closed.get();
return indexService.closed.get() == false;
}
}

Expand Down
3 changes: 2 additions & 1 deletion server/src/main/java/org/elasticsearch/node/Node.java
Expand Up @@ -501,7 +501,8 @@ protected Node(

final PersistentTasksExecutorRegistry registry = new PersistentTasksExecutorRegistry(tasksExecutors);
final PersistentTasksClusterService persistentTasksClusterService =
new PersistentTasksClusterService(settings, registry, clusterService);
new PersistentTasksClusterService(settings, registry, clusterService, threadPool);
resourcesToClose.add(persistentTasksClusterService);
final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client);

modules.add(b -> {
Expand Down

0 comments on commit 13cb0fb

Please sign in to comment.