Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 11 additions & 1 deletion core/trino-main/src/main/java/io/trino/split/SplitManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,11 @@
import io.trino.spi.connector.Constraint;
import io.trino.spi.connector.DynamicFilter;
import io.trino.tracing.TrinoAttributes;
import jakarta.annotation.PreDestroy;

import java.util.Optional;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;

import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.SystemSessionProperties.isAllowPushdownIntoConnectors;
Expand All @@ -44,6 +46,7 @@ public class SplitManager
private final CatalogServiceProvider<ConnectorSplitManager> splitManagerProvider;
private final Tracer tracer;
private final int minScheduleSplitBatchSize;
private final ExecutorService executorService;
private final Executor executor;

@Inject
Expand All @@ -52,7 +55,14 @@ public SplitManager(CatalogServiceProvider<ConnectorSplitManager> splitManagerPr
this.splitManagerProvider = requireNonNull(splitManagerProvider, "splitManagerProvider is null");
this.tracer = requireNonNull(tracer, "tracer is null");
this.minScheduleSplitBatchSize = config.getMinScheduleSplitBatchSize();
this.executor = new BoundedExecutor(newCachedThreadPool(daemonThreadsNamed("splits-manager-callback-%s")), config.getMaxSplitManagerCallbackThreads());
this.executorService = newCachedThreadPool(daemonThreadsNamed("splits-manager-callback-%s"));
this.executor = new BoundedExecutor(executorService, config.getMaxSplitManagerCallbackThreads());
}

@PreDestroy
public void shutdown()
{
executorService.shutdown();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the benefit of doing this? The underlying pool uses daemon threads, so they'll all terminate when the VM stops. shutdown() does not wait for or interrupt the threads, so it won't do much except for preventing new tasks from being submitted.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In a typical deployment situation, this has no effect since the SplitManager is created only once. However, in tests this ensures that the threadpool is shutdown and the threads stopped earlier than they would if the usual cached threadpool 60 second idle timeout is reached.

}

public SplitSource getSplits(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,14 @@
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.concurrent.BoundedExecutor;
import io.trino.split.SplitSource.SplitBatch;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.parallel.Execution;
import org.junit.jupiter.api.parallel.ExecutionMode;

import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;

import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.MoreFutures.getFutureValue;
Expand All @@ -29,13 +34,22 @@
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;

@TestInstance(TestInstance.Lifecycle.PER_CLASS)
@Execution(ExecutionMode.CONCURRENT)
public class TestBufferingSplitSource
{
private static final Executor executor = new BoundedExecutor(newCachedThreadPool(daemonThreadsNamed(TestBufferingSplitSource.class.getSimpleName() + "-%s")), 10);
private final ExecutorService executorService = newCachedThreadPool(daemonThreadsNamed(TestBufferingSplitSource.class.getSimpleName() + "-%s"));
private final Executor executor = new BoundedExecutor(executorService, 10);

@AfterAll
public void cleanup()
{
executorService.shutdown();
}

@Test
public void testSlowSource()
Expand Down