Skip to content

Commit

Permalink
Fix compile errors and test failures
Browse files Browse the repository at this point in the history
  • Loading branch information
zhicwu committed May 19, 2022
1 parent 45c95d3 commit 92e075d
Show file tree
Hide file tree
Showing 13 changed files with 163 additions and 67 deletions.
29 changes: 22 additions & 7 deletions clickhouse-cli-client/README.md
Original file line number Diff line number Diff line change
@@ -1,14 +1,18 @@
# ClickHouse Command-line Client

This is a wrapper of ClickHouse native command-line client. In order to use it, please make sure 1) either the native command-line client or docker is installed; and 2) `clickhouse_cli_path` or `docker_cli_path` is configured properly.
This is a thin wrapper of ClickHouse native command-line client. It provides an alternative way to communicate with ClickHouse, which might be of use when you prefer:

Unlike `clickhouse-http-client`, this module is not designed for dealing with many queries in short period of time, because it uses sub-process(NOT thread) and file-based streaming. Having said that, it provides an alternative, usually faster,way to dump and load large data sets. Besides, due to its simplicity, it can be used as an example to demonstrate how to implement SPI defined in `clickhouse-client`.
- TCP/native protocol over HTTP or gRPC
- 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.

## Limitations and Known Issues

- Only `max_result_rows` and `result_overflow_mode` two settings are currently supported
- ClickHouseResponseSummary is always empty - see ClickHouse/ClickHouse#37241
- Session is not supported and query cannot be cancelled - see ClickHouse/ClickHouse#37308
- Session is not supported - see ClickHouse/ClickHouse#37308

## Maven Dependency

Expand All @@ -27,7 +31,7 @@ Unlike `clickhouse-http-client`, this module is not designed for dealing with ma
// make sure 'clickhouse-client' 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 -Dclickhouse_cli_path=/path/to/clickhouse-client -Ddocker_cli_path=/path/to/docker MyProgram
// java -Dchc_clickhouse_cli_path=/path/to/clickhouse-client -Dchc_docker_cli_path=/path/to/docker MyProgram

// clickhouse-cli-client uses TCP protocol
ClickHouseProtocol preferredProtocol = ClickHouseProtocol.TCP;
Expand All @@ -37,13 +41,13 @@ ClickHouseNode server = ClickHouseNode.builder().host("my-server").port(preferre
// declares a file
ClickHouseFile file = ClickHouseFile.of("data.csv");

// dump query results into the file - format is TSV, according to file extension
// dump query results into the file - format is CSV, according to file extension
ClickHouseClient.dump(server, "select * from some_table", file).get();

// now load it into my_table, using TSV format
// now load it into my_table, using CSV format
ClickHouseClient.load(server, "my_table", file).get();

// it can be used in the same as any other client
// it can be used in the same way as any other client
try (ClickHouseClient client = ClickHouseClient.newInstance(preferredProtocol);
ClickHouseResponse response = client.connect(server)
.query("select * from numbers(:limit)")
Expand All @@ -53,4 +57,15 @@ try (ClickHouseClient client = ClickHouseClient.newInstance(preferredProtocol);
String str = r.getValue(0).asString();
}
}

// and of course it's part of JDBC driver
try (Connection conn = DriverManager.getConnect("jdbc:ch:tcp://my-server", "default", "");
PreparedStatement stmt = conn.preparedStatement("select * from numbers(?)")) {
stmt.setInt(1, 1000);
ResultSet rs = stmt.executeQuery();
while (rs.next()) {
int num = rs.getInt(1);
String str = rs.getString(1);
}
}
```
30 changes: 15 additions & 15 deletions clickhouse-cli-client/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,12 @@
<groupId>${project.parent.groupId}</groupId>
<artifactId>clickhouse-client</artifactId>
<version>${revision}</version>
</dependency>
<dependency>
<groupId>org.lz4</groupId>
<artifactId>lz4-java</artifactId>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- necessary for Java 9+ -->
Expand All @@ -54,11 +56,6 @@
<artifactId>testcontainers</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.github.tomakehurst</groupId>
<artifactId>wiremock-jre8</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
Expand All @@ -83,12 +80,6 @@
<createDependencyReducedPom>true</createDependencyReducedPom>
<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
<shadedClassifierName>shaded</shadedClassifierName>
<relocations>
<relocation>
<pattern>net.jpountz</pattern>
<shadedPattern>${shade.base}.jpountz</shadedPattern>
</relocation>
</relocations>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer" />
Expand Down Expand Up @@ -116,6 +107,15 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<configuration>
<environmentVariables>
<CHC_TEST_CONTAINER_ID>clickhouse-cli-client</CHC_TEST_CONTAINER_ID>
</environmentVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,10 @@
import java.io.IOException;
import java.io.OutputStream;
import java.io.UncheckedIOException;
import java.lang.ProcessBuilder.Redirect;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedList;
Expand Down Expand Up @@ -60,7 +60,7 @@ static boolean check(int timeout, String command, String... args) {
Collections.addAll(list, args);
Process process = null;
try {
process = new ProcessBuilder(list).redirectError(Redirect.DISCARD).start();
process = new ProcessBuilder(list).start();
process.getOutputStream().close();
if (process.waitFor(timeout, TimeUnit.MILLISECONDS)) {
int exitValue = process.exitValue();
Expand Down Expand Up @@ -168,7 +168,7 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest<?> request)
}
if ((boolean) config.getOption(ClickHouseCommandLineOption.USE_CLI_CONFIG)) {
str = (String) config.getOption(ClickHouseCommandLineOption.CLI_CONFIG_FILE);
if (Files.exists(Path.of(str))) {
if (Files.exists(Paths.get(str))) {
commands.add("--config-file=".concat(str));
}
} else {
Expand All @@ -194,13 +194,13 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest<?> request)
if (!tableFile.isAvailable() || !tableFile.getFile().getAbsolutePath().startsWith(hostDir)) {
// creating a hard link is faster but it's not platform-independent
File f = ClickHouseInputStream.save(
Path.of(hostDir, "chc_".concat(UUID.randomUUID().toString())).toFile(),
Paths.get(hostDir, "chc_".concat(UUID.randomUUID().toString())).toFile(),
table.getContent(), config.getWriteBufferSize(), config.getSocketTimeout(), true);
filePath = containerDir.concat(f.getName());
} else {
filePath = tableFile.getFile().getAbsolutePath();
if (!hostDir.equals(containerDir)) {
filePath = Path.of(containerDir, filePath.substring(hostDir.length())).toFile().getAbsolutePath();
filePath = Paths.get(containerDir, filePath.substring(hostDir.length())).toFile().getAbsolutePath();
}
}
commands.add("--file=" + filePath);
Expand Down Expand Up @@ -235,7 +235,7 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest<?> request)
String workDirectory = (String) config.getOption(
ClickHouseCommandLineOption.CLI_WORK_DIRECTORY);
if (!ClickHouseChecker.isNullOrBlank(workDirectory)) {
Path p = Path.of(workDirectory);
Path p = Paths.get(workDirectory);
if (Files.isDirectory(p)) {
builder.directory(p.toFile());
}
Expand Down Expand Up @@ -264,7 +264,7 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest<?> request)
fileName = new StringBuilder(len + uuid.length() + 1).append(fileName).append('_')
.append(UUID.randomUUID().toString()).toString();
}
Path newPath = Path.of(hostDir, fileName);
Path newPath = Paths.get(hostDir, fileName);
try {
f = Files.createLink(newPath, f.toPath()).toFile();
} catch (IOException e) {
Expand Down
Original file line number Diff line number Diff line change
@@ -1,14 +1,26 @@
package com.clickhouse.client.cli;

import com.clickhouse.client.ClickHouseClient;
import com.clickhouse.client.ClickHouseClientBuilder;
import com.clickhouse.client.ClickHouseNode;
import com.clickhouse.client.ClickHouseProtocol;
import com.clickhouse.client.ClickHouseServerForTest;
import com.clickhouse.client.ClientIntegrationTest;
import com.clickhouse.client.cli.config.ClickHouseCommandLineOption;

import org.testcontainers.containers.GenericContainer;
import org.testng.Assert;
import org.testng.SkipException;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

public class ClickHouseCommandLineClientTest extends ClientIntegrationTest {
@BeforeClass
static void init() {
System.setProperty(ClickHouseCommandLineOption.CLI_CONTAINER_DIRECTORY.getSystemProperty(),
ClickHouseServerForTest.getClickHouseContainerTmpDir());
}

@Override
protected ClickHouseProtocol getProtocol() {
return ClickHouseProtocol.TCP;
Expand All @@ -19,9 +31,25 @@ protected Class<? extends ClickHouseClient> getClientClass() {
return ClickHouseCommandLineClient.class;
}

@Override
protected ClickHouseClientBuilder initClient(ClickHouseClientBuilder builder) {
return super.initClient(builder).option(ClickHouseCommandLineOption.CLI_CONTAINER_DIRECTORY,
ClickHouseServerForTest.getClickHouseContainerTmpDir());
}

@Override
protected ClickHouseNode getServer() {
GenericContainer<?> container = ClickHouseServerForTest.getClickHouseContainer();
if (container != null) {
return ClickHouseNode.of("localhost", getProtocol(), getProtocol().getDefaultPort(), null);
}

return super.getServer();
}

@Test(groups = { "integration" })
@Override
public void testLoadRawData() {
public void testLoadRawData() throws Exception {
throw new SkipException("Skip due to response summary is always empty");
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
package com.clickhouse.client;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.UncheckedIOException;
Expand Down Expand Up @@ -305,12 +303,10 @@ static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server,
* @throws IllegalArgumentException if any of server, tableOrQuery, and output
* is null
* @throws CompletionException when error occurred during execution
* @throws IOException when failed to create the file or its parent
* directories
*/
static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server, String tableOrQuery,
ClickHouseFormat format, ClickHouseCompression compression, String file) throws IOException {
return dump(server, tableOrQuery, format, compression, ClickHouseUtils.getFileOutputStream(file));
ClickHouseFormat format, ClickHouseCompression compression, String file) {
return dump(server, tableOrQuery, ClickHouseFile.of(file, compression, 0, format));
}

/**
Expand Down Expand Up @@ -403,11 +399,10 @@ static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server,
* @return future object to get result
* @throws IllegalArgumentException if any of server, table, and input is null
* @throws CompletionException when error occurred during execution
* @throws FileNotFoundException when file not found
*/
static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server, String table,
ClickHouseFormat format, ClickHouseCompression compression, String file) throws FileNotFoundException {
return load(server, table, format, compression, ClickHouseUtils.getFileInputStream(file));
ClickHouseFormat format, ClickHouseCompression compression, String file) {
return load(server, table, ClickHouseFile.of(file, compression, 0, format));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,16 @@ public static ClickHouseInputStream of(ClickHouseDeferredValue<InputStream> defe
return new WrappedInputStream(null, new DeferredInputStream(deferredInput), bufferSize, postCloseAction);
}

/**
* Wraps the given file as input stream.
*
* @param file non-null file
* @param bufferSize buffer size which is always greater than zero(usually
* 8192 or larger)
* @param postCloseAction custom action will be performed right after closing
* the input stream
* @return wrapped input
*/
public static ClickHouseInputStream of(ClickHouseFile file, int bufferSize, Runnable postCloseAction) {
if (file == null || !file.isAvailable()) {
throw new IllegalArgumentException("Non-null file required");
Expand Down Expand Up @@ -220,7 +230,7 @@ public static ClickHouseInputStream of(InputStream input, int bufferSize, ClickH
Runnable postCloseAction) {
if (input == null) {
return EmptyInputStream.INSTANCE;
} else if (input instanceof ClickHouseInputStream) {
} else if (input != EmptyInputStream.INSTANCE && input instanceof ClickHouseInputStream) {
return (ClickHouseInputStream) input;
}
return wrap(null, input, bufferSize, postCloseAction, compression, 0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,17 @@ public static ClickHouseOutputStream empty() {
return EmptyOutputStream.INSTANCE;
}

/**
* Wraps the given file as output stream.
*
* @param file non-null file
* @param postCloseAction custom action will be performed right after closing
* the output stream
* @param bufferSize buffer size which is always greater than zero(usually
* 8192
* or larger)
* @return wrapped output
*/
public static ClickHouseOutputStream of(ClickHouseFile file, int bufferSize, Runnable postCloseAction) {
if (file == null || file == ClickHouseFile.NULL) {
throw new IllegalArgumentException("Non-null file required");
Expand All @@ -85,7 +96,7 @@ public static ClickHouseOutputStream of(ClickHouseFile file, int bufferSize, Run
/**
* Wraps the given output stream.
*
* @param output non-null output stream
* @param output output stream
* @return wrapped output, or the same output if it's instance of
* {@link ClickHouseOutputStream}
*/
Expand All @@ -96,7 +107,7 @@ public static ClickHouseOutputStream of(OutputStream output) {
/**
* Wraps the given output stream.
*
* @param output non-null output stream
* @param output output stream
* @param bufferSize buffer size which is always greater than zero(usually 8192
* or larger)
* @return wrapped output, or the same output if it's instance of
Expand All @@ -109,7 +120,7 @@ public static ClickHouseOutputStream of(OutputStream output, int bufferSize) {
/**
* Wraps the given output stream.
*
* @param output non-null output stream
* @param output output stream
* @param bufferSize buffer size which is always greater than zero(usually
* 8192 or larger)
* @param compression compression algorithm, null or
Expand All @@ -123,8 +134,11 @@ public static ClickHouseOutputStream of(OutputStream output, int bufferSize) {
public static ClickHouseOutputStream of(OutputStream output, int bufferSize, ClickHouseCompression compression,
Runnable postCloseAction) {
final ClickHouseOutputStream chOutput;
if (compression == null || compression == ClickHouseCompression.NONE) {
chOutput = output instanceof ClickHouseOutputStream ? (ClickHouseOutputStream) output
if (output == null) {
chOutput = EmptyOutputStream.INSTANCE;
} else if (compression == null || compression == ClickHouseCompression.NONE) {
chOutput = output != EmptyOutputStream.INSTANCE && output instanceof ClickHouseOutputStream
? (ClickHouseOutputStream) output
: new WrappedOutputStream(null, output, bufferSize, postCloseAction);
} else {
chOutput = wrap(null, output, bufferSize, postCloseAction, compression, 0);
Expand Down
Loading

0 comments on commit 92e075d

Please sign in to comment.