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:
Gian Merlino 2024-10-22 20:21:21 -07:00 committed by GitHub
parent 1157ecdec3
commit 60daddedf8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
8 changed files with 773 additions and 309 deletions

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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(),

View File

@ -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

View File

@ -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();