diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 7dc1322a366..e9bba479c64 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -544,6 +544,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer announceTask(task, zkWorker, taskRunnerWorkItem); return true; } else { + log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); return false; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java index c5bdc254193..abc4da0ad57 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java @@ -160,7 +160,7 @@ public class ZkWorker implements Closeable public ImmutableZkWorker toImmutable() { - return new ImmutableZkWorker(this); + return new ImmutableZkWorker(worker, getCurrCapacityUsed(), getAvailabilityGroups()); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/FillCapacityWorkerSelectStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/FillCapacityWorkerSelectStrategy.java index 59476b46e6d..db305136d43 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/FillCapacityWorkerSelectStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/FillCapacityWorkerSelectStrategy.java @@ -24,10 +24,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.inject.Inject; -import com.metamx.emitter.EmittingLogger; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.ImmutableZkWorker; -import io.druid.indexing.overlord.ZkWorker; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import java.util.Comparator; @@ -37,8 +35,6 @@ import java.util.TreeSet; */ public class FillCapacityWorkerSelectStrategy implements WorkerSelectStrategy { - private static final EmittingLogger log = new EmittingLogger(FillCapacityWorkerSelectStrategy.class); - private final RemoteTaskRunnerConfig config; @Inject @@ -77,7 +73,6 @@ public class FillCapacityWorkerSelectStrategy implements WorkerSelectStrategy return Optional.of(zkWorker); } } - log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); return Optional.absent(); }