mirror of https://github.com/apache/druid.git
SeekableStreamSupervisor: Use workerExec as the client connectExec. (#17394)
* SeekableStreamSupervisor: Use workerExec as the client connectExec. This patch uses the already-existing per-supervisor workerExec as the connectExec for task clients, rather than using the process-wide default ServiceClientFactory pool. This helps prevent callbacks from backlogging on the process-wide pool. It's especially useful for retries, where callbacks may need to establish new TCP connections or perform TLS handshakes. * Fix compilation, tests. * Fix style.
This commit is contained in:
parent
1157ecdec3
commit
60daddedf8
|
@ -25,17 +25,17 @@ import org.apache.druid.guice.LazySingleton;
|
|||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
|
||||
@LazySingleton
|
||||
public class RabbitStreamIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, Long>
|
||||
{
|
||||
@Inject
|
||||
public RabbitStreamIndexTaskClientFactory(
|
||||
@EscalatedGlobal ServiceClientFactory serviceClientFactory,
|
||||
@EscalatedGlobal HttpClient httpClient,
|
||||
@Json ObjectMapper mapper)
|
||||
{
|
||||
super(serviceClientFactory, mapper);
|
||||
super(httpClient, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,18 +26,18 @@ import org.apache.druid.guice.LazySingleton;
|
|||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
|
||||
@LazySingleton
|
||||
public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KafkaTopicPartition, Long>
|
||||
{
|
||||
@Inject
|
||||
public KafkaIndexTaskClientFactory(
|
||||
@EscalatedGlobal ServiceClientFactory serviceClientFactory,
|
||||
@EscalatedGlobal HttpClient httpClient,
|
||||
@Json ObjectMapper mapper
|
||||
)
|
||||
{
|
||||
super(serviceClientFactory, mapper);
|
||||
super(httpClient, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -25,18 +25,18 @@ import org.apache.druid.guice.LazySingleton;
|
|||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
|
||||
@LazySingleton
|
||||
public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, String>
|
||||
{
|
||||
@Inject
|
||||
public KinesisIndexTaskClientFactory(
|
||||
@EscalatedGlobal ServiceClientFactory serviceClientFactory,
|
||||
@EscalatedGlobal HttpClient httpClient,
|
||||
@Json ObjectMapper mapper
|
||||
)
|
||||
{
|
||||
super(serviceClientFactory, mapper);
|
||||
super(httpClient, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -111,6 +111,7 @@ import java.util.Set;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
public class KinesisSupervisorTest extends EasyMockSupport
|
||||
{
|
||||
|
@ -5117,11 +5118,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig,
|
||||
ScheduledExecutorService connectExec
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
|
@ -5259,14 +5259,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig,
|
||||
ScheduledExecutorService connectExec
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(
|
||||
replicas * (autoScalerConfig != null ? autoScalerConfig.getTaskCountMax() : taskCount),
|
||||
maxNumTasks
|
||||
);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
|
@ -5348,11 +5344,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig,
|
||||
ScheduledExecutorService connectExec
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
|
@ -5436,11 +5431,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
public SeekableStreamIndexTaskClient<String, String> build(
|
||||
String dataSource,
|
||||
TaskInfoProvider taskInfoProvider,
|
||||
int maxNumTasks,
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
SeekableStreamSupervisorTuningConfig tuningConfig,
|
||||
ScheduledExecutorService connectExec
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(replicas * taskCount, maxNumTasks);
|
||||
Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
|
||||
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
|
||||
return taskClient;
|
||||
|
|
|
@ -21,31 +21,46 @@ package org.apache.druid.indexing.seekablestream;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
|
||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.rpc.ServiceClientFactoryImpl;
|
||||
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
public abstract class SeekableStreamIndexTaskClientFactory<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
private static final Logger log = new Logger(SeekableStreamIndexTaskClientFactory.class);
|
||||
|
||||
private final ServiceClientFactory serviceClientFactory;
|
||||
private final HttpClient httpClient;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
protected SeekableStreamIndexTaskClientFactory(
|
||||
final ServiceClientFactory serviceClientFactory,
|
||||
final HttpClient httpClient,
|
||||
final ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
this.serviceClientFactory = serviceClientFactory;
|
||||
this.httpClient = httpClient;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a task client for a specific supervisor.
|
||||
*
|
||||
* @param dataSource task datasource
|
||||
* @param taskInfoProvider task locator
|
||||
* @param tuningConfig from {@link SeekableStreamSupervisor#tuningConfig}
|
||||
* @param connectExec should generally be {@link SeekableStreamSupervisor#workerExec}. This is preferable to
|
||||
* the global pool for the default {@link ServiceClientFactory}, to prevent callbacks from
|
||||
* different supervisors from backlogging each other.
|
||||
*/
|
||||
public SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> build(
|
||||
final String dataSource,
|
||||
final TaskInfoProvider taskInfoProvider,
|
||||
final int maxNumTasks,
|
||||
final SeekableStreamSupervisorTuningConfig tuningConfig
|
||||
final SeekableStreamSupervisorTuningConfig tuningConfig,
|
||||
final ScheduledExecutorService connectExec
|
||||
)
|
||||
{
|
||||
log.info(
|
||||
|
@ -57,7 +72,7 @@ public abstract class SeekableStreamIndexTaskClientFactory<PartitionIdType, Sequ
|
|||
|
||||
return new SeekableStreamIndexTaskClientAsyncImpl<PartitionIdType, SequenceOffsetType>(
|
||||
dataSource,
|
||||
serviceClientFactory,
|
||||
new ServiceClientFactoryImpl(httpClient, connectExec),
|
||||
taskInfoProvider,
|
||||
jsonMapper,
|
||||
tuningConfig.getHttpTimeout(),
|
||||
|
|
|
@ -33,7 +33,7 @@ import com.google.common.collect.ImmutableSet;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectLinkedOpenHashMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
|
@ -65,7 +65,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata
|
|||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientAsyncImpl;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||
|
@ -86,6 +85,7 @@ import org.apache.druid.java.util.common.RetryUtils;
|
|||
import org.apache.druid.java.util.common.Stopwatch;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
|
@ -837,12 +837,25 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
private final SeekableStreamIndexTaskTuningConfig taskTuningConfig;
|
||||
private final String supervisorId;
|
||||
private final TaskInfoProvider taskInfoProvider;
|
||||
private final long futureTimeoutInSeconds; // how long to wait for async operations to complete
|
||||
private final RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
/**
|
||||
* Single-threaded executor for running {@link Notice#handle()} from {@link #notices}.
|
||||
*/
|
||||
private final ExecutorService exec;
|
||||
/**
|
||||
* Single-threaded scheduled executor for adding periodic {@link RunNotice} to {@link #notices}.
|
||||
*/
|
||||
private final ScheduledExecutorService scheduledExec;
|
||||
/**
|
||||
* Single-threaded scheduled executor for reporting metircs on notice queue size, lag, etc.
|
||||
* See {@link #scheduleReporting}.
|
||||
*/
|
||||
private final ScheduledExecutorService reportingExec;
|
||||
private final ListeningExecutorService workerExec;
|
||||
/**
|
||||
* Multi-threaded executor for managing communications with workers, including handling callbacks from worker RPCs.
|
||||
* Also serves as the connectExec for {@link #taskClient}.
|
||||
*/
|
||||
private final ListeningScheduledExecutorService workerExec;
|
||||
private final NoticesQueue<Notice> notices = new NoticesQueue<>();
|
||||
private final Object stopLock = new Object();
|
||||
private final Object stateChangeLock = new Object();
|
||||
|
@ -903,21 +916,16 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
);
|
||||
|
||||
int workerThreads;
|
||||
int maxNumTasks;
|
||||
if (autoScalerConfig != null && autoScalerConfig.getEnableTaskAutoScaler()) {
|
||||
log.info("Running Task autoscaler for datasource [%s]", dataSource);
|
||||
|
||||
workerThreads = (this.tuningConfig.getWorkerThreads() != null
|
||||
? this.tuningConfig.getWorkerThreads()
|
||||
: Math.min(10, autoScalerConfig.getTaskCountMax()));
|
||||
|
||||
maxNumTasks = autoScalerConfig.getTaskCountMax() * this.ioConfig.getReplicas();
|
||||
} else {
|
||||
workerThreads = (this.tuningConfig.getWorkerThreads() != null
|
||||
? this.tuningConfig.getWorkerThreads()
|
||||
: Math.min(10, this.ioConfig.getTaskCount()));
|
||||
|
||||
maxNumTasks = this.ioConfig.getTaskCount() * this.ioConfig.getReplicas();
|
||||
}
|
||||
|
||||
IdleConfig specIdleConfig = spec.getIoConfig().getIdleConfig();
|
||||
|
@ -938,7 +946,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
|
||||
this.workerExec = MoreExecutors.listeningDecorator(
|
||||
Execs.multiThreaded(
|
||||
ScheduledExecutors.fixed(
|
||||
workerThreads,
|
||||
StringUtils.encodeForFormat(supervisorId) + "-Worker-%d"
|
||||
)
|
||||
|
@ -973,13 +981,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
};
|
||||
|
||||
this.futureTimeoutInSeconds = Math.max(
|
||||
MINIMUM_FUTURE_TIMEOUT_IN_SECONDS,
|
||||
tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds()
|
||||
+ SeekableStreamIndexTaskClientAsyncImpl.MAX_RETRY_WAIT_SECONDS)
|
||||
);
|
||||
|
||||
this.taskClient = taskClientFactory.build(dataSource, taskInfoProvider, maxNumTasks, this.tuningConfig);
|
||||
this.taskClient = taskClientFactory.build(dataSource, taskInfoProvider, this.tuningConfig, workerExec);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -179,7 +179,7 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
|||
EasyMock.expect(taskClientFactory.build(
|
||||
EasyMock.anyString(),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyInt(),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyObject()
|
||||
)).andReturn(
|
||||
indexTaskClient).anyTimes();
|
||||
|
|
Loading…
Reference in New Issue