Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[client-v2] Do not close provided executor #2088

Merged
merged 1 commit into from
Jan 15, 2025
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
23 changes: 16 additions & 7 deletions client-v2/src/main/java/com/clickhouse/client/api/Client.java
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,8 @@ public class Client implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(Client.class);
private final ExecutorService sharedOperationExecutor;

private final boolean isSharedOpExecuterorOwned;

private final Map<String, ClientStatisticsHolder> globalClientStats = new ConcurrentHashMap<>();

private boolean useNewImplementation = false;
Expand Down Expand Up @@ -172,8 +174,10 @@ private Client(Set<String> endpoints, Map<String,String> configuration, boolean

boolean isAsyncEnabled = MapUtils.getFlag(this.configuration, ClientConfigProperties.ASYNC_OPERATIONS.getKey(), false);
if (isAsyncEnabled && sharedOperationExecutor == null) {
this.isSharedOpExecuterorOwned = true;
this.sharedOperationExecutor = Executors.newCachedThreadPool(new DefaultThreadFactory("chc-operation"));
} else {
this.isSharedOpExecuterorOwned = false;
this.sharedOperationExecutor = sharedOperationExecutor;
}
this.useNewImplementation = useNewImplementation;
Expand Down Expand Up @@ -225,12 +229,16 @@ public String getDefaultDatabase() {
*/
@Override
public void close() {
try {
if (sharedOperationExecutor != null && !sharedOperationExecutor.isShutdown()) {
this.sharedOperationExecutor.shutdownNow();
if (isSharedOpExecuterorOwned) {
try {
if (sharedOperationExecutor != null && !sharedOperationExecutor.isShutdown()) {
this.sharedOperationExecutor.shutdownNow();
}
} catch (Exception e) {
LOG.error("Failed to close shared operation executor", e);
}
} catch (Exception e) {
LOG.error("Failed to close shared operation executor", e);
} else {
LOG.debug("Skip closing operation executor because not owned by client");
}

if (oldClient != null) {
Expand Down Expand Up @@ -777,9 +785,10 @@ public Builder useAsyncRequests(boolean async) {
/**
* Sets an executor for running operations. If async operations are enabled and no executor is specified
* client will create a default executor.
*
* Executor will stay running after {@code Client#close() } is called. It is application responsibility to close
* the executor.
* @param executorService - executor service for async operations
* @return
* @return
*/
public Builder setSharedOperationExecutor(ExecutorService executorService) {
this.sharedOperationExecutor = executorService;
Expand Down
23 changes: 23 additions & 0 deletions client-v2/src/test/java/com/clickhouse/client/ClientTests.java
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.enums.Protocol;
import com.clickhouse.client.api.query.GenericRecord;
import com.clickhouse.client.api.query.QueryResponse;
import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.client.api.query.Records;
import com.clickhouse.client.config.ClickHouseClientOption;
Expand All @@ -19,7 +20,10 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;

public class ClientTests extends BaseIntegrationTest {
private static final Logger LOGGER = LoggerFactory.getLogger(ClientTests.class);
Expand Down Expand Up @@ -130,6 +134,25 @@ public void testSetOptions() {
}
}

@Test
public void testProvidedExecutor() throws Exception {

ExecutorService executorService = Executors.newSingleThreadExecutor();
try (Client client = newClient().useAsyncRequests(true).setSharedOperationExecutor(executorService).build()) {
QueryResponse response = client.query("SELECT 1").get();
response.getMetrics();
} catch (Exception e) {
Assert.fail("unexpected exception", e);
}

AtomicBoolean flag = new AtomicBoolean(true);
executorService.submit(() -> flag.compareAndSet(true, false));
executorService.shutdown();
executorService.awaitTermination(10, TimeUnit.SECONDS);

Assert.assertFalse(flag.get());
}

protected Client.Builder newClient() {
ClickHouseNode node = getServer(ClickHouseProtocol.HTTP);
boolean isSecure = isCloud();
Expand Down
Loading