Skip to content

Commit

Permalink
Fix slowness in performance mode and failover not working when protoc…
Browse files Browse the repository at this point in the history
…ol is unsupported
  • Loading branch information
zhicwu committed Jul 13, 2022
1 parent 9b21161 commit 04b9c4b
Show file tree
Hide file tree
Showing 8 changed files with 90 additions and 27 deletions.
10 changes: 5 additions & 5 deletions clickhouse-cli-client/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,11 @@ This is a thin wrapper of ClickHouse native command-line client. It provides an
- native CLI client instead of pure Java implementation
- an example of implementing SPI defined in `clickhouse-client` module

Either [clickhouse-client](https://clickhouse.com/docs/en/interfaces/cli/) or [docker](https://docs.docker.com/get-docker/) must be installed prior to use. And it's important to understand that this module uses sub-process(in addition to threads) and file-based streaming, meaning 1) it's not as fast as native CLI client or pure Java implementation, although it's close in the case of dumping and loading data; and 2) it's not suitable for scenarios like dealing with many queries in short period of time.
Either [clickhouse](https://clickhouse.com/docs/en/interfaces/cli/) or [docker](https://docs.docker.com/get-docker/) must be installed prior to use. And it's important to understand that this module uses sub-process(in addition to threads) and file-based streaming, meaning 1) it's not as fast as native CLI client or pure Java implementation, although it's close in the case of dumping and loading data; and 2) it's not suitable for scenarios like dealing with many queries in short period of time.

## Limitations and Known Issues

- Only `max_result_rows` and `result_overflow_mode` two settings are currently supported
- Only `max_result_rows`, `result_overflow_mode` and `readonly` 3 settings are currently supported
- ClickHouseResponseSummary is always empty - see ClickHouse/ClickHouse#37241
- Session is not supported - see ClickHouse/ClickHouse#37308

Expand All @@ -28,10 +28,10 @@ Either [clickhouse-client](https://clickhouse.com/docs/en/interfaces/cli/) or [d
## Examples

```java
// make sure 'clickhouse-client' or 'docker' is in PATH before you start the program
// make sure 'clickhouse' or 'docker' is in PATH before you start the program
// alternatively, configure CLI path in either Java system property or environment variable, for examples:
// CHC_CLICKHOUSE_CLI_PATH=/path/to/clickhouse-client CHC_DOCKER_CLI_PATH=/path/to/docker java MyProgram
// java -Dchc_clickhouse_cli_path=/path/to/clickhouse-client -Dchc_docker_cli_path=/path/to/docker MyProgram
// CHC_CLICKHOUSE_CLI_PATH=/path/to/clickhouse CHC_DOCKER_CLI_PATH=/path/to/docker java MyProgram
// java -Dchc_clickhouse_cli_path=/path/to/clickhouse -Dchc_docker_cli_path=/path/to/docker MyProgram

// clickhouse-cli-client uses TCP protocol
ClickHouseProtocol preferredProtocol = ClickHouseProtocol.TCP;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
import java.io.IOException;
import java.io.OutputStream;
import java.io.UncheckedIOException;
import java.net.ConnectException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
Expand Down Expand Up @@ -93,7 +94,7 @@ static void dockerCommand(ClickHouseConfig config, String hostDir, String contai
cli = DEFAULT_DOCKER_CLI_PATH;
}
if (!check(timeout, cli, DEFAULT_CLI_ARG_VERSION)) {
throw new IllegalStateException("Docker command-line is not available: " + cli);
throw new UncheckedIOException(new ConnectException("Docker command-line is not available: " + cli));
} else {
commands.add(cli);
}
Expand All @@ -111,7 +112,7 @@ static void dockerCommand(ClickHouseConfig config, String hostDir, String contai
DEFAULT_CLI_ARG_VERSION)
&& !check(timeout, cli, "run", "--rm", "--name", str, "-v", hostDir + ':' + containerDir,
"-d", img, "tail", "-f", "/dev/null")) {
throw new IllegalStateException("Failed to start new container: " + str);
throw new UncheckedIOException(new ConnectException("Failed to start new container: " + str));
}
}
}
Expand All @@ -122,7 +123,7 @@ static void dockerCommand(ClickHouseConfig config, String hostDir, String contai
} else { // create new container for each query
if (!check(timeout, cli, "run", "--rm", img, DEFAULT_CLICKHOUSE_CLI_PATH, DEFAULT_CLIENT_OPTION,
DEFAULT_CLI_ARG_VERSION)) {
throw new IllegalStateException("Invalid ClickHouse docker image: " + img);
throw new UncheckedIOException(new ConnectException("Invalid ClickHouse docker image: " + img));
}
commands.add("run");
commands.add("--rm");
Expand Down Expand Up @@ -235,6 +236,10 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest<?> request)
if (value != null) {
commands.add("--result_overflow_mode=".concat(value.toString()));
}
value = settings.get("readonly");
if (value != null) {
commands.add("--readonly=".concat(value.toString()));
}
if ((boolean) config.getOption(ClickHouseCommandLineOption.USE_PROFILE_EVENTS)) {
commands.add("--print-profile-events");
commands.add("--profile-events-delay-ms=-1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,10 @@
public enum ClickHouseCommandLineOption implements ClickHouseOption {
/**
* ClickHouse native command-line client path. Empty value is treated as
* 'clickhouse-client'.
* 'clickhouse'.
*/
CLICKHOUSE_CLI_PATH("clickhouse_cli_path", "",
"ClickHouse native command-line client path, empty value is treated as 'clickhouse-client'"),
"ClickHouse native command-line client path, empty value is treated as 'clickhouse'"),
/**
* ClickHouse docker image. Empty value is treated as
* 'clickhouse/clickhouse-server'.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ static ClickHouseInputStream getResponseInputStream(ClickHouseConfig config, Inp

/**
* Gets piped input stream for reading data from response asynchronously. When
* {@code config} is null or {@code config.isAsync()} is faluse, this method is
* {@code config} is null or {@code config.isAsync()} is false, this method is
* same as
* {@link #getResponseInputStream(ClickHouseConfig, InputStream, Runnable)}.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
package com.clickhouse.client;

import java.io.Serializable;
import java.io.UncheckedIOException;
import java.net.ConnectException;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
Expand All @@ -27,7 +29,7 @@
*/
public class ClickHouseClientBuilder {
/**
* Dummy client which is only used {@link Agent}.
* Dummy client which is only used by {@link Agent}.
*/
static class DummyClient implements ClickHouseClient {
static final ClickHouseConfig CONFIG = new ClickHouseConfig();
Expand All @@ -40,7 +42,10 @@ public boolean accept(ClickHouseProtocol protocol) {

@Override
public CompletableFuture<ClickHouseResponse> execute(ClickHouseRequest<?> request) {
return CompletableFuture.completedFuture(ClickHouseResponse.EMPTY);
CompletableFuture<ClickHouseResponse> future = new CompletableFuture<>();
future.completeExceptionally(
new ConnectException("No client available for connecting to: " + request.getServer()));
return future;
}

@Override
Expand All @@ -55,7 +60,7 @@ public void close() {

@Override
public boolean ping(ClickHouseNode server, int timeout) {
return true;
return false;
}
}

Expand Down Expand Up @@ -166,6 +171,11 @@ ClickHouseResponse retry(ClickHouseRequest<?> sealedRequest, Throwable cause, in
}

ClickHouseResponse handle(ClickHouseRequest<?> sealedRequest, Throwable cause) {
// in case there's any recoverable exception wrapped by UncheckedIOException
if (cause instanceof UncheckedIOException && cause.getCause() != null) {
cause = ((UncheckedIOException) cause).getCause();
}

try {
int times = sealedRequest.getConfig().getFailover();
if (times > 0) {
Expand Down Expand Up @@ -352,7 +362,7 @@ public ClickHouseClient build() {
}
}

if (client == null) {
if (client == null && !agent) {
throw new IllegalStateException(
ClickHouseUtils.format("No suitable ClickHouse client(out of %d) found in classpath for %s.",
counter, nodeSelector));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -314,6 +314,10 @@ public static ClickHouseNodes of(String enpoints, Map<?, ?> options) {
* Load balancing tags for filtering out nodes.
*/
protected final ClickHouseNodeSelector selector;
/**
* Flag indicating whether it's single node or not.
*/
protected final boolean singleNode;
/**
* Template node.
*/
Expand Down Expand Up @@ -358,8 +362,11 @@ protected ClickHouseNodes(Collection<ClickHouseNode> nodes, ClickHouseNode templ
n.setManager(this);
}
if (autoDiscovery) {
this.singleNode = false;
this.discoveryFuture.getAndUpdate(current -> policy.schedule(current, ClickHouseNodes.this::discover,
(int) template.config.getOption(ClickHouseClientOption.NODE_DISCOVERY_INTERVAL)));
} else {
this.singleNode = nodes.size() == 1;
}
this.healthCheckFuture.getAndUpdate(current -> policy.schedule(current, ClickHouseNodes.this::check,
(int) template.config.getOption(ClickHouseClientOption.HEALTH_CHECK_INTERVAL)));
Expand Down Expand Up @@ -472,6 +479,15 @@ protected ClickHouseNode get() {
return apply(selector);
}

/**
* Checks whether it's single node or not.
*
* @return true if it's single node; false otherwise
*/
public boolean isSingleNode() {
return singleNode;
}

@Override
public ClickHouseNode apply(ClickHouseNodeSelector t) {
lock.readLock().lock();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,13 @@

import com.clickhouse.client.ClickHouseChecker;
import com.clickhouse.client.ClickHouseClient;
import com.clickhouse.client.ClickHouseClientBuilder;
import com.clickhouse.client.ClickHouseColumn;
import com.clickhouse.client.ClickHouseConfig;
import com.clickhouse.client.ClickHouseFormat;
import com.clickhouse.client.ClickHouseNode;
import com.clickhouse.client.ClickHouseNodeSelector;
import com.clickhouse.client.ClickHouseNodes;
import com.clickhouse.client.ClickHouseParameterizedQuery;
import com.clickhouse.client.ClickHouseRecord;
import com.clickhouse.client.ClickHouseRequest;
Expand Down Expand Up @@ -218,16 +220,33 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException {
jdbcConf = connInfo.getJdbcConfig();

autoCommit = !jdbcConf.isJdbcCompliant() || jdbcConf.isAutoCommit();

ClickHouseNode node = connInfo.getServer();
log.debug("Connecting to: %s", node);

jvmTimeZone = TimeZone.getDefault();

client = ClickHouseClient.builder().options(ClickHouseDriver.toClientOptions(connInfo.getProperties()))
.defaultCredentials(connInfo.getDefaultCredentials())
.nodeSelector(ClickHouseNodeSelector.of(node.getProtocol())).build();
clientRequest = client.connect(node);
ClickHouseClientBuilder clientBuilder = ClickHouseClient.builder()
.options(ClickHouseDriver.toClientOptions(connInfo.getProperties()))
.defaultCredentials(connInfo.getDefaultCredentials());
ClickHouseNodes nodes = connInfo.getNodes();
final ClickHouseNode node;
if (nodes.isSingleNode()) {
try {
node = nodes.apply(nodes.getNodeSelector());
} catch (Exception e) {
throw SqlExceptionUtils.clientError("Failed to get single-node", e);
}
client = clientBuilder.nodeSelector(ClickHouseNodeSelector.of(node.getProtocol())).build();
clientRequest = client.connect(node);
} else {
log.debug("Selecting node from: %s", nodes);
client = clientBuilder.nodeSelector(nodes.getNodeSelector()).build();
clientRequest = client.connect(nodes);
try {
node = clientRequest.getServer();
} catch (Exception e) {
throw SqlExceptionUtils.clientError("No healthy node available", e);
}
}

log.debug("Connecting to: %s", node);
ClickHouseConfig config = clientRequest.getConfig();
String currentUser = null;
TimeZone timeZone = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,10 @@
import com.clickhouse.client.ClickHouseUtils;
import com.clickhouse.client.config.ClickHouseClientOption;
import com.clickhouse.client.config.ClickHouseDefaults;
import com.clickhouse.client.logging.Logger;
import com.clickhouse.client.logging.LoggerFactory;
import com.clickhouse.jdbc.JdbcConfig;
import com.clickhouse.jdbc.SqlExceptionUtils;

public class ClickHouseJdbcUrlParser {
private static final Logger log = LoggerFactory.getLogger(ClickHouseJdbcUrlParser.class);

public static class ConnectionInfo {
private final ClickHouseCredentials credentials;
private final ClickHouseNodes nodes;
Expand All @@ -46,6 +42,14 @@ public ClickHouseCredentials getDefaultCredentials() {
return this.credentials;
}

/**
* Gets selected server.
*
* @return non-null selected server
* @deprecated will be removed in v0.3.3, please use {@link #getNodes()}
* instead
*/
@Deprecated
public ClickHouseNode getServer() {
return nodes.apply(nodes.getNodeSelector());
}
Expand All @@ -54,6 +58,15 @@ public JdbcConfig getJdbcConfig() {
return jdbcConf;
}

/**
* Gets nodes defined in connection string.
*
* @return non-null nodes
*/
public ClickHouseNodes getNodes() {
return nodes;
}

public Properties getProperties() {
return props;
}
Expand Down

0 comments on commit 04b9c4b

Please sign in to comment.