mirror of https://github.com/apache/druid.git
don't hold thread while waiting after failure from server (#4795)
This commit is contained in:
parent
b61248fdb1
commit
d37be5e6e9
|
@ -81,7 +81,7 @@ public class RetryUtils
|
||||||
return retry(f, shouldRetry, 0, maxTries);
|
return retry(f, shouldRetry, 0, maxTries);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void awaitNextRetry(Throwable e, final int nTry, final boolean quiet) throws InterruptedException
|
private static void awaitNextRetry(final Throwable e, final int nTry, final boolean quiet) throws InterruptedException
|
||||||
{
|
{
|
||||||
|
|
||||||
final long sleepMillis = nextRetrySleepMillis(nTry);
|
final long sleepMillis = nextRetrySleepMillis(nTry);
|
||||||
|
@ -95,7 +95,7 @@ public class RetryUtils
|
||||||
Thread.sleep(sleepMillis);
|
Thread.sleep(sleepMillis);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static long nextRetrySleepMillis(final int nTry) throws InterruptedException
|
public static long nextRetrySleepMillis(final int nTry)
|
||||||
{
|
{
|
||||||
final long baseSleepMillis = 1000;
|
final long baseSleepMillis = 1000;
|
||||||
final long maxSleepMillis = 60000;
|
final long maxSleepMillis = 60000;
|
||||||
|
|
|
@ -31,7 +31,6 @@ import com.google.common.net.HostAndPort;
|
||||||
import com.google.common.util.concurrent.FutureCallback;
|
import com.google.common.util.concurrent.FutureCallback;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
|
@ -50,6 +49,7 @@ import io.druid.java.util.common.ISE;
|
||||||
import io.druid.java.util.common.Pair;
|
import io.druid.java.util.common.Pair;
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.java.util.common.concurrent.ScheduledExecutors;
|
||||||
import io.druid.java.util.common.lifecycle.LifecycleStart;
|
import io.druid.java.util.common.lifecycle.LifecycleStart;
|
||||||
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
||||||
import io.druid.server.coordination.DataSegmentChangeRequest;
|
import io.druid.server.coordination.DataSegmentChangeRequest;
|
||||||
|
@ -75,9 +75,8 @@ import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
import java.util.concurrent.LinkedBlockingDeque;
|
import java.util.concurrent.LinkedBlockingDeque;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
|
@ -103,7 +102,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
// For each queryable server, a name -> DruidServerHolder entry is kept
|
// For each queryable server, a name -> DruidServerHolder entry is kept
|
||||||
private final ConcurrentHashMap<String, DruidServerHolder> servers = new ConcurrentHashMap<>();
|
private final ConcurrentHashMap<String, DruidServerHolder> servers = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
private volatile ExecutorService executor;
|
private volatile ScheduledExecutorService executor;
|
||||||
|
|
||||||
// the work queue, all items in this are sequentially processed by main thread setup in start()
|
// the work queue, all items in this are sequentially processed by main thread setup in start()
|
||||||
// used to call inventoryInitialized on all SegmentCallbacks and
|
// used to call inventoryInitialized on all SegmentCallbacks and
|
||||||
|
@ -145,9 +144,9 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
log.info("Starting HttpServerInventoryView.");
|
log.info("Starting HttpServerInventoryView.");
|
||||||
|
|
||||||
try {
|
try {
|
||||||
executor = Executors.newFixedThreadPool(
|
executor = ScheduledExecutors.fixed(
|
||||||
config.getNumThreads(),
|
config.getNumThreads(),
|
||||||
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("HttpServerInventoryView-%s").build()
|
"HttpServerInventoryView-%s"
|
||||||
);
|
);
|
||||||
|
|
||||||
executor.execute(
|
executor.execute(
|
||||||
|
@ -433,10 +432,23 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
private volatile long unstableStartTime = -1;
|
private volatile long unstableStartTime = -1;
|
||||||
private volatile int consecutiveFailedAttemptCount = 0;
|
private volatile int consecutiveFailedAttemptCount = 0;
|
||||||
|
|
||||||
|
private final Runnable addToQueueRunnable;
|
||||||
|
|
||||||
DruidServerHolder(DruidServer druidServer)
|
DruidServerHolder(DruidServer druidServer)
|
||||||
{
|
{
|
||||||
this.druidServer = druidServer;
|
this.druidServer = druidServer;
|
||||||
this.serverHostAndPort = HostAndPort.fromString(druidServer.getHost());
|
this.serverHostAndPort = HostAndPort.fromString(druidServer.getHost());
|
||||||
|
|
||||||
|
this.addToQueueRunnable = () -> {
|
||||||
|
queue.add(
|
||||||
|
() -> {
|
||||||
|
DruidServerHolder holder = servers.get(druidServer.getName());
|
||||||
|
if (holder != null) {
|
||||||
|
holder.updateSegmentsListAsync();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
//wait for first fetch of segment listing from server.
|
//wait for first fetch of segment listing from server.
|
||||||
|
@ -571,10 +583,21 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
consecutiveFailedAttemptCount = 0;
|
consecutiveFailedAttemptCount = 0;
|
||||||
}
|
}
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
log.error(ex, "error processing segment list response from server [%s]", druidServer.getName());
|
String logMsg = StringUtils.nonStrictFormat(
|
||||||
|
"Error processing segment list response from server [%s]. Reason [%s]",
|
||||||
|
druidServer.getName(),
|
||||||
|
ex.getMessage()
|
||||||
|
);
|
||||||
|
|
||||||
|
if (incrementFailedAttemptAndCheckUnstabilityTimeout()) {
|
||||||
|
log.error(ex, logMsg);
|
||||||
|
} else {
|
||||||
|
log.info("Temporary Failure. %s", logMsg);
|
||||||
|
log.debug(ex, logMsg);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
addNextSyncToWorkQueue(druidServer.getName());
|
addNextSyncToWorkQueue();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -603,17 +626,9 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
log.debug(logMsg);
|
log.debug(logMsg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// sleep for a bit so that retry does not happen immediately.
|
|
||||||
try {
|
|
||||||
Thread.sleep(RetryUtils.nextRetrySleepMillis(consecutiveFailedAttemptCount));
|
|
||||||
}
|
|
||||||
catch (InterruptedException ex) {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
addNextSyncToWorkQueue(druidServer.getName());
|
addNextSyncToWorkQueue();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
@ -632,17 +647,9 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
log.info("Temporary Failure. %s", logMsg);
|
log.info("Temporary Failure. %s", logMsg);
|
||||||
log.debug(th, logMsg);
|
log.debug(th, logMsg);
|
||||||
}
|
}
|
||||||
|
|
||||||
// sleep for a bit so that retry does not happen immediately.
|
|
||||||
try {
|
|
||||||
Thread.sleep(RetryUtils.nextRetrySleepMillis(consecutiveFailedAttemptCount));
|
|
||||||
}
|
|
||||||
catch (InterruptedException ex) {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
addNextSyncToWorkQueue(druidServer.getName());
|
addNextSyncToWorkQueue();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -696,16 +703,28 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addNextSyncToWorkQueue(final String serverId)
|
private void addNextSyncToWorkQueue()
|
||||||
{
|
{
|
||||||
queue.add(
|
if (consecutiveFailedAttemptCount > 0) {
|
||||||
() -> {
|
try {
|
||||||
DruidServerHolder holder = servers.get(serverId);
|
long sleepMillis = RetryUtils.nextRetrySleepMillis(consecutiveFailedAttemptCount);
|
||||||
if (holder != null) {
|
log.info("Scheduling next syncup in [%d] millis from server [%s].", sleepMillis, druidServer.getName());
|
||||||
holder.updateSegmentsListAsync();
|
executor.schedule(
|
||||||
}
|
addToQueueRunnable,
|
||||||
}
|
sleepMillis,
|
||||||
);
|
TimeUnit.MILLISECONDS
|
||||||
|
);
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
log.makeAlert(
|
||||||
|
ex,
|
||||||
|
"WTF! Couldn't schedule next sync. Server[%s] is not being synced any more, restarting Druid process on that server might fix the issue.",
|
||||||
|
druidServer.getName()
|
||||||
|
).emit();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
addToQueueRunnable.run();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean incrementFailedAttemptAndCheckUnstabilityTimeout()
|
private boolean incrementFailedAttemptAndCheckUnstabilityTimeout()
|
||||||
|
|
Loading…
Reference in New Issue