mirror of https://github.com/apache/druid.git
Removing unused exec service. (#13541)
This commit is contained in:
parent
7002ecd303
commit
5a3d79a5d5
|
@ -38,12 +38,10 @@ import org.apache.druid.indexing.common.task.AbstractTask;
|
|||
import org.apache.druid.indexing.common.task.Tasks;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.msq.exec.Controller;
|
||||
import org.apache.druid.msq.exec.ControllerContext;
|
||||
import org.apache.druid.msq.exec.ControllerImpl;
|
||||
import org.apache.druid.msq.exec.MSQTasks;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.rpc.StandardRetryPolicy;
|
||||
import org.apache.druid.rpc.indexing.OverlordClient;
|
||||
|
@ -52,8 +50,6 @@ import org.joda.time.Interval;
|
|||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
@JsonTypeName(MSQControllerTask.TYPE)
|
||||
public class MSQControllerTask extends AbstractTask
|
||||
|
@ -74,8 +70,6 @@ public class MSQControllerTask extends AbstractTask
|
|||
// Enables users, and the web console, to see the original SQL type names (if any). Not used by any other Druid logic.
|
||||
@Nullable
|
||||
private final List<String> sqlTypeNames;
|
||||
@Nullable
|
||||
private final ExecutorService remoteFetchExecutorService;
|
||||
|
||||
// Using an Injector directly because tasks do not have a way to provide their own Guice modules.
|
||||
@JacksonInject
|
||||
|
@ -106,13 +100,6 @@ public class MSQControllerTask extends AbstractTask
|
|||
this.sqlQueryContext = sqlQueryContext;
|
||||
this.sqlTypeNames = sqlTypeNames;
|
||||
|
||||
if (MultiStageQueryContext.isDurableStorageEnabled(querySpec.getQuery().context())) {
|
||||
this.remoteFetchExecutorService =
|
||||
Executors.newCachedThreadPool(Execs.makeThreadFactory(getId() + "-remote-fetcher-%d"));
|
||||
} else {
|
||||
this.remoteFetchExecutorService = null;
|
||||
}
|
||||
|
||||
addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true);
|
||||
}
|
||||
|
||||
|
@ -198,10 +185,6 @@ public class MSQControllerTask extends AbstractTask
|
|||
if (controller != null) {
|
||||
controller.stopGracefully();
|
||||
}
|
||||
if (remoteFetchExecutorService != null) {
|
||||
// This is to make sure we don't leak connections.
|
||||
remoteFetchExecutorService.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue