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.EscalatedGlobal;
import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Json;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.java.util.http.client.HttpClient;
@LazySingleton @LazySingleton
public class RabbitStreamIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, Long> public class RabbitStreamIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, Long>
{ {
@Inject @Inject
public RabbitStreamIndexTaskClientFactory( public RabbitStreamIndexTaskClientFactory(
@EscalatedGlobal ServiceClientFactory serviceClientFactory, @EscalatedGlobal HttpClient httpClient,
@Json ObjectMapper mapper) @Json ObjectMapper mapper)
{ {
super(serviceClientFactory, mapper); super(httpClient, mapper);
} }
@Override @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.EscalatedGlobal;
import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Json;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.java.util.http.client.HttpClient;
@LazySingleton @LazySingleton
public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KafkaTopicPartition, Long> public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KafkaTopicPartition, Long>
{ {
@Inject @Inject
public KafkaIndexTaskClientFactory( public KafkaIndexTaskClientFactory(
@EscalatedGlobal ServiceClientFactory serviceClientFactory, @EscalatedGlobal HttpClient httpClient,
@Json ObjectMapper mapper @Json ObjectMapper mapper
) )
{ {
super(serviceClientFactory, mapper); super(httpClient, mapper);
} }
@Override @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.EscalatedGlobal;
import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Json;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.java.util.http.client.HttpClient;
@LazySingleton @LazySingleton
public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, String> public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<String, String>
{ {
@Inject @Inject
public KinesisIndexTaskClientFactory( public KinesisIndexTaskClientFactory(
@EscalatedGlobal ServiceClientFactory serviceClientFactory, @EscalatedGlobal HttpClient httpClient,
@Json ObjectMapper mapper @Json ObjectMapper mapper
) )
{ {
super(serviceClientFactory, mapper); super(httpClient, mapper);
} }
@Override @Override

View File

@ -111,6 +111,7 @@ import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
public class KinesisSupervisorTest extends EasyMockSupport public class KinesisSupervisorTest extends EasyMockSupport
{ {
@ -5117,11 +5118,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
public SeekableStreamIndexTaskClient<String, String> build( public SeekableStreamIndexTaskClient<String, String> build(
String dataSource, String dataSource,
TaskInfoProvider taskInfoProvider, 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_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
return taskClient; return taskClient;
@ -5259,14 +5259,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
public SeekableStreamIndexTaskClient<String, String> build( public SeekableStreamIndexTaskClient<String, String> build(
String dataSource, String dataSource,
TaskInfoProvider taskInfoProvider, 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_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
return taskClient; return taskClient;
@ -5348,11 +5344,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
public SeekableStreamIndexTaskClient<String, String> build( public SeekableStreamIndexTaskClient<String, String> build(
String dataSource, String dataSource,
TaskInfoProvider taskInfoProvider, 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_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
return taskClient; return taskClient;
@ -5436,11 +5431,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
public SeekableStreamIndexTaskClient<String, String> build( public SeekableStreamIndexTaskClient<String, String> build(
String dataSource, String dataSource,
TaskInfoProvider taskInfoProvider, 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_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout());
Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries());
return taskClient; return taskClient;

View File

@ -21,31 +21,46 @@ package org.apache.druid.indexing.seekablestream;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.indexing.common.TaskInfoProvider; 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.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
import org.apache.druid.java.util.common.logger.Logger; 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.ServiceClientFactory;
import org.apache.druid.rpc.ServiceClientFactoryImpl;
import java.util.concurrent.ScheduledExecutorService;
public abstract class SeekableStreamIndexTaskClientFactory<PartitionIdType, SequenceOffsetType> public abstract class SeekableStreamIndexTaskClientFactory<PartitionIdType, SequenceOffsetType>
{ {
private static final Logger log = new Logger(SeekableStreamIndexTaskClientFactory.class); private static final Logger log = new Logger(SeekableStreamIndexTaskClientFactory.class);
private final ServiceClientFactory serviceClientFactory; private final HttpClient httpClient;
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
protected SeekableStreamIndexTaskClientFactory( protected SeekableStreamIndexTaskClientFactory(
final ServiceClientFactory serviceClientFactory, final HttpClient httpClient,
final ObjectMapper jsonMapper final ObjectMapper jsonMapper
) )
{ {
this.serviceClientFactory = serviceClientFactory; this.httpClient = httpClient;
this.jsonMapper = jsonMapper; 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( public SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> build(
final String dataSource, final String dataSource,
final TaskInfoProvider taskInfoProvider, final TaskInfoProvider taskInfoProvider,
final int maxNumTasks, final SeekableStreamSupervisorTuningConfig tuningConfig,
final SeekableStreamSupervisorTuningConfig tuningConfig final ScheduledExecutorService connectExec
) )
{ {
log.info( log.info(
@ -57,7 +72,7 @@ public abstract class SeekableStreamIndexTaskClientFactory<PartitionIdType, Sequ
return new SeekableStreamIndexTaskClientAsyncImpl<PartitionIdType, SequenceOffsetType>( return new SeekableStreamIndexTaskClientAsyncImpl<PartitionIdType, SequenceOffsetType>(
dataSource, dataSource,
serviceClientFactory, new ServiceClientFactoryImpl(httpClient, connectExec),
taskInfoProvider, taskInfoProvider,
jsonMapper, jsonMapper,
tuningConfig.getHttpTimeout(), tuningConfig.getHttpTimeout(),

View File

@ -33,7 +33,7 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
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.ListeningExecutorService; import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import it.unimi.dsi.fastutil.ints.Int2ObjectLinkedOpenHashMap; import it.unimi.dsi.fastutil.ints.Int2ObjectLinkedOpenHashMap;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap; 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.SeekableStreamEndSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; 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.SeekableStreamIndexTaskClientFactory;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; 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.Stopwatch;
import org.apache.druid.java.util.common.StringUtils; 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.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.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; 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 SeekableStreamIndexTaskTuningConfig taskTuningConfig;
private final String supervisorId; private final String supervisorId;
private final TaskInfoProvider taskInfoProvider; private final TaskInfoProvider taskInfoProvider;
private final long futureTimeoutInSeconds; // how long to wait for async operations to complete
private final RowIngestionMetersFactory rowIngestionMetersFactory; private final RowIngestionMetersFactory rowIngestionMetersFactory;
/**
* Single-threaded executor for running {@link Notice#handle()} from {@link #notices}.
*/
private final ExecutorService exec; private final ExecutorService exec;
/**
* Single-threaded scheduled executor for adding periodic {@link RunNotice} to {@link #notices}.
*/
private final ScheduledExecutorService scheduledExec; 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 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 NoticesQueue<Notice> notices = new NoticesQueue<>();
private final Object stopLock = new Object(); private final Object stopLock = new Object();
private final Object stateChangeLock = new Object(); private final Object stateChangeLock = new Object();
@ -903,21 +916,16 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
); );
int workerThreads; int workerThreads;
int maxNumTasks;
if (autoScalerConfig != null && autoScalerConfig.getEnableTaskAutoScaler()) { if (autoScalerConfig != null && autoScalerConfig.getEnableTaskAutoScaler()) {
log.info("Running Task autoscaler for datasource [%s]", dataSource); log.info("Running Task autoscaler for datasource [%s]", dataSource);
workerThreads = (this.tuningConfig.getWorkerThreads() != null workerThreads = (this.tuningConfig.getWorkerThreads() != null
? this.tuningConfig.getWorkerThreads() ? this.tuningConfig.getWorkerThreads()
: Math.min(10, autoScalerConfig.getTaskCountMax())); : Math.min(10, autoScalerConfig.getTaskCountMax()));
maxNumTasks = autoScalerConfig.getTaskCountMax() * this.ioConfig.getReplicas();
} else { } else {
workerThreads = (this.tuningConfig.getWorkerThreads() != null workerThreads = (this.tuningConfig.getWorkerThreads() != null
? this.tuningConfig.getWorkerThreads() ? this.tuningConfig.getWorkerThreads()
: Math.min(10, this.ioConfig.getTaskCount())); : Math.min(10, this.ioConfig.getTaskCount()));
maxNumTasks = this.ioConfig.getTaskCount() * this.ioConfig.getReplicas();
} }
IdleConfig specIdleConfig = spec.getIoConfig().getIdleConfig(); IdleConfig specIdleConfig = spec.getIoConfig().getIdleConfig();
@ -938,7 +946,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
} }
this.workerExec = MoreExecutors.listeningDecorator( this.workerExec = MoreExecutors.listeningDecorator(
Execs.multiThreaded( ScheduledExecutors.fixed(
workerThreads, workerThreads,
StringUtils.encodeForFormat(supervisorId) + "-Worker-%d" StringUtils.encodeForFormat(supervisorId) + "-Worker-%d"
) )
@ -973,13 +981,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
} }
}; };
this.futureTimeoutInSeconds = Math.max( this.taskClient = taskClientFactory.build(dataSource, taskInfoProvider, this.tuningConfig, workerExec);
MINIMUM_FUTURE_TIMEOUT_IN_SECONDS,
tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds()
+ SeekableStreamIndexTaskClientAsyncImpl.MAX_RETRY_WAIT_SECONDS)
);
this.taskClient = taskClientFactory.build(dataSource, taskInfoProvider, maxNumTasks, this.tuningConfig);
} }
@Override @Override

View File

@ -179,7 +179,7 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
EasyMock.expect(taskClientFactory.build( EasyMock.expect(taskClientFactory.build(
EasyMock.anyString(), EasyMock.anyString(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyInt(), EasyMock.anyObject(),
EasyMock.anyObject() EasyMock.anyObject()
)).andReturn( )).andReturn(
indexTaskClient).anyTimes(); indexTaskClient).anyTimes();