Skip to content

Commit 3a53cf1

Browse files
authored
Merge pull request #1200 from zhicwu/master
consistent behavior of setting InputStream and ClickHouseWriter in ClickHouseRequest
2 parents 6e689ad + 84e0301 commit 3a53cf1

File tree

15 files changed

+370
-155
lines changed

15 files changed

+370
-155
lines changed

clickhouse-benchmark/pom.xml

+39-14
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,6 @@
1-
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
1+
<project xmlns="http://maven.apache.org/POM/4.0.0"
2+
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
3+
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
24
<modelVersion>4.0.0</modelVersion>
35

46
<parent>
@@ -83,6 +85,38 @@
8385
</exclusion>
8486
</exclusions>
8587
</dependency>
88+
<dependency>
89+
<groupId>com.aayushatharva.brotli4j</groupId>
90+
<artifactId>brotli4j</artifactId>
91+
</dependency>
92+
<dependency>
93+
<groupId>com.aayushatharva.brotli4j</groupId>
94+
<artifactId>native-linux-x86_64</artifactId>
95+
</dependency>
96+
<dependency>
97+
<groupId>com.aayushatharva.brotli4j</groupId>
98+
<artifactId>native-osx-aarch64</artifactId>
99+
</dependency>
100+
<dependency>
101+
<groupId>com.aayushatharva.brotli4j</groupId>
102+
<artifactId>native-osx-x86_64</artifactId>
103+
</dependency>
104+
<dependency>
105+
<groupId>com.aayushatharva.brotli4j</groupId>
106+
<artifactId>native-windows-x86_64</artifactId>
107+
</dependency>
108+
<dependency>
109+
<groupId>com.github.luben</groupId>
110+
<artifactId>zstd-jni</artifactId>
111+
</dependency>
112+
<dependency>
113+
<groupId>org.lz4</groupId>
114+
<artifactId>lz4-java</artifactId>
115+
</dependency>
116+
<dependency>
117+
<groupId>org.tukaani</groupId>
118+
<artifactId>xz</artifactId>
119+
</dependency>
86120
<dependency>
87121
<groupId>org.jctools</groupId>
88122
<artifactId>jctools-core</artifactId>
@@ -119,23 +153,14 @@
119153
<configuration>
120154
<finalName>${shade.name}</finalName>
121155
<transformers>
122-
<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
156+
<transformer
157+
implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
123158
<mainClass>org.openjdk.jmh.Main</mainClass>
124159
</transformer>
125-
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
160+
<transformer
161+
implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
126162
</transformers>
127163
<filters>
128-
<filter>
129-
<artifact>com.clickhouse:clickhouse-jdbc</artifact>
130-
<excludes>
131-
<exclude>**/module-info.class</exclude>
132-
<exclude>ru/**</exclude>
133-
<exclude>META-INF/MANIFEST.MF</exclude>
134-
<exclude>META-INF/*.SF</exclude>
135-
<exclude>META-INF/*.DSA</exclude>
136-
<exclude>META-INF/*.RSA</exclude>
137-
</excludes>
138-
</filter>
139164
<filter>
140165
<artifact>*:*</artifact>
141166
<excludes>

clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLine.java

+4-1
Original file line numberDiff line numberDiff line change
@@ -193,6 +193,9 @@ static Process startProcess(ClickHouseRequest<?> request) {
193193
final ClickHouseNode server = request.getServer();
194194
final int timeout = config.getSocketTimeout();
195195

196+
// FIXME potential timing issue
197+
final Optional<ClickHouseInputStream> in = request.getInputStream();
198+
196199
String hostDir = config.getStrOption(ClickHouseCommandLineOption.CLI_WORK_DIRECTORY);
197200
hostDir = ClickHouseUtils.normalizeDirectory(
198201
ClickHouseChecker.isNullOrBlank(hostDir) ? System.getProperty("java.io.tmpdir") : hostDir);
@@ -382,7 +385,7 @@ static Process startProcess(ClickHouseRequest<?> request) {
382385
builder.redirectOutput(f);
383386
}
384387
}
385-
final Optional<ClickHouseInputStream> in = request.getInputStream();
388+
386389
try {
387390
final Process process;
388391
if (in.isPresent()) {

clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineResponse.java

+2
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,8 @@
88
import com.clickhouse.client.ClickHouseStreamResponse;
99

1010
public class ClickHouseCommandLineResponse extends ClickHouseStreamResponse {
11+
private static final long serialVersionUID = 4253185543390807162L;
12+
1113
private final transient ClickHouseCommandLine cli;
1214

1315
protected ClickHouseCommandLineResponse(ClickHouseConfig config, ClickHouseCommandLine cli) throws IOException {

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java

+49-88
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,6 @@
44
import java.io.InputStream;
55
import java.io.OutputStream;
66
import java.io.Serializable;
7-
import java.io.UncheckedIOException;
87
import java.util.ArrayList;
98
import java.util.Collection;
109
import java.util.Collections;
@@ -17,12 +16,8 @@
1716
import java.util.Map;
1817
import java.util.Objects;
1918
import java.util.Map.Entry;
20-
import java.util.concurrent.CancellationException;
2119
import java.util.concurrent.CompletableFuture;
2220
import java.util.concurrent.CompletionException;
23-
import java.util.concurrent.ExecutionException;
24-
import java.util.concurrent.TimeUnit;
25-
import java.util.concurrent.TimeoutException;
2621
import java.util.concurrent.atomic.AtomicReference;
2722
import java.util.Optional;
2823
import java.util.Properties;
@@ -31,20 +26,17 @@
3126
import java.util.function.Function;
3227

3328
import com.clickhouse.client.config.ClickHouseClientOption;
34-
import com.clickhouse.config.ClickHouseBufferingMode;
3529
import com.clickhouse.config.ClickHouseConfigChangeListener;
3630
import com.clickhouse.config.ClickHouseOption;
3731
import com.clickhouse.data.ClickHouseChecker;
3832
import com.clickhouse.data.ClickHouseCompression;
39-
import com.clickhouse.data.ClickHouseDataStreamFactory;
4033
import com.clickhouse.data.ClickHouseDeferredValue;
4134
import com.clickhouse.data.ClickHouseExternalTable;
4235
import com.clickhouse.data.ClickHouseFile;
4336
import com.clickhouse.data.ClickHouseFormat;
4437
import com.clickhouse.data.ClickHouseInputStream;
4538
import com.clickhouse.data.ClickHouseOutputStream;
4639
import com.clickhouse.data.ClickHousePassThruStream;
47-
import com.clickhouse.data.ClickHousePipedOutputStream;
4840
import com.clickhouse.data.ClickHouseUtils;
4941
import com.clickhouse.data.ClickHouseValue;
5042
import com.clickhouse.data.ClickHouseValues;
@@ -69,12 +61,26 @@ public class ClickHouseRequest<SelfT extends ClickHouseRequest<SelfT>> implement
6961
SPECIAL_SETTINGS = Collections.unmodifiableSet(set);
7062
}
7163

64+
static class PipedWriter implements ClickHouseWriter {
65+
private final ClickHouseDeferredValue<ClickHouseInputStream> input;
66+
67+
PipedWriter(ClickHouseDeferredValue<ClickHouseInputStream> input) {
68+
this.input = input;
69+
}
70+
71+
@Override
72+
public void write(ClickHouseOutputStream output) throws IOException {
73+
ClickHouseInputStream in = input.get();
74+
if (in != null) {
75+
in.pipe(output);
76+
}
77+
}
78+
}
79+
7280
/**
7381
* Mutation request.
7482
*/
7583
public static class Mutation extends ClickHouseRequest<Mutation> {
76-
private transient ClickHouseWriter writer;
77-
7884
protected Mutation(ClickHouseRequest<?> request, boolean sealed) {
7985
super(request.getClient(), request.server, request.serverRef, request.options, sealed);
8086

@@ -151,16 +157,18 @@ public Mutation format(ClickHouseFormat format) {
151157
}
152158

153159
/**
154-
* Sets custom writer for streaming. This will create a piped stream between the
155-
* writer and ClickHouse server.
160+
* Sets custom writer for streaming. This will remove input stream set by other
161+
* {@code data()} methods.
156162
*
157163
* @param writer writer
158164
* @return mutation request
159165
*/
160166
public Mutation data(ClickHouseWriter writer) {
161167
checkSealed();
162168

169+
this.input = changeProperty(PROP_DATA, this.input, null);
163170
this.writer = changeProperty(PROP_WRITER, this.writer, writer);
171+
164172
return this;
165173
}
166174

@@ -191,6 +199,7 @@ public Mutation data(ClickHousePassThruStream stream) {
191199
final int bufferSize = c.getReadBufferSize();
192200
this.input = changeProperty(PROP_DATA, this.input, ClickHouseDeferredValue
193201
.of(() -> ClickHouseInputStream.of(stream, bufferSize, null)));
202+
this.writer = changeProperty(PROP_WRITER, this.writer, null);
194203
return this;
195204
}
196205

@@ -259,6 +268,7 @@ public Mutation data(ClickHouseInputStream input) {
259268

260269
this.input = changeProperty(PROP_DATA, this.input,
261270
ClickHouseDeferredValue.of(input, ClickHouseInputStream.class));
271+
this.writer = changeProperty(PROP_WRITER, this.writer, null);
262272
return this;
263273
}
264274

@@ -272,83 +282,11 @@ public Mutation data(ClickHouseDeferredValue<ClickHouseInputStream> input) {
272282
checkSealed();
273283

274284
this.input = changeProperty(PROP_DATA, this.input, input);
285+
this.writer = changeProperty(PROP_WRITER, this.writer, null);
275286

276287
return this;
277288
}
278289

279-
@Override
280-
public CompletableFuture<ClickHouseResponse> execute() {
281-
if (writer != null) {
282-
final ClickHouseConfig c = getConfig();
283-
final boolean perfMode = c.getResponseBuffering() == ClickHouseBufferingMode.PERFORMANCE;
284-
final ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance()
285-
.createPipedOutputStream(c, null);
286-
data(stream.getInputStream());
287-
CompletableFuture<ClickHouseResponse> future = null;
288-
if (c.isAsync()) {
289-
future = getClient().execute(this);
290-
}
291-
292-
if (perfMode) {
293-
ClickHouseClient.submit(() -> {
294-
295-
});
296-
} else {
297-
try (ClickHouseOutputStream out = stream) {
298-
writer.write(out);
299-
} catch (IOException e) {
300-
throw new CompletionException(e);
301-
}
302-
}
303-
304-
if (future != null) {
305-
return future;
306-
}
307-
}
308-
309-
return getClient().execute(this);
310-
}
311-
312-
@Override
313-
public ClickHouseResponse executeAndWait() throws ClickHouseException {
314-
if (writer != null) {
315-
ClickHouseConfig c = getConfig();
316-
ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance()
317-
.createPipedOutputStream(c, null);
318-
data(stream.getInputStream());
319-
CompletableFuture<ClickHouseResponse> future = null;
320-
if (c.isAsync()) {
321-
future = getClient().execute(this);
322-
}
323-
try (ClickHouseOutputStream out = stream) {
324-
writer.write(out);
325-
} catch (IOException e) {
326-
throw ClickHouseException.of(e, getServer());
327-
}
328-
if (future != null) {
329-
try {
330-
return future.get(c.getSocketTimeout(), TimeUnit.MILLISECONDS);
331-
} catch (InterruptedException e) {
332-
Thread.currentThread().interrupt();
333-
throw ClickHouseException.forCancellation(e, getServer());
334-
} catch (CancellationException e) {
335-
throw ClickHouseException.forCancellation(e, getServer());
336-
} catch (ExecutionException | TimeoutException | UncheckedIOException e) {
337-
Throwable cause = e.getCause();
338-
if (cause == null) {
339-
cause = e;
340-
}
341-
throw cause instanceof ClickHouseException ? (ClickHouseException) cause
342-
: ClickHouseException.of(cause, getServer());
343-
} catch (RuntimeException e) { // unexpected
344-
throw ClickHouseException.of(e, getServer());
345-
}
346-
}
347-
}
348-
349-
return getClient().executeAndWait(this);
350-
}
351-
352290
@Override
353291
public Mutation table(String table, String queryId) {
354292
checkSealed();
@@ -369,6 +307,7 @@ public Mutation seal() {
369307
req.namedParameters.putAll(namedParameters);
370308

371309
req.input = input;
310+
req.writer = writer;
372311
req.queryId = queryId;
373312
req.sql = sql;
374313

@@ -407,6 +346,7 @@ public Mutation seal() {
407346

408347
protected final Map<String, String> namedParameters;
409348

349+
protected transient ClickHouseWriter writer;
410350
protected transient ClickHouseDeferredValue<ClickHouseInputStream> input;
411351
protected transient ClickHouseDeferredValue<ClickHouseOutputStream> output;
412352
protected String queryId;
@@ -508,6 +448,7 @@ public ClickHouseRequest<SelfT> copy() {
508448
req.settings.putAll(settings);
509449
req.namedParameters.putAll(namedParameters);
510450
req.input = input;
451+
req.writer = writer;
511452
req.output = output;
512453
req.queryId = queryId;
513454
req.sql = sql;
@@ -555,12 +496,12 @@ public boolean isTransactional() {
555496
}
556497

557498
/**
558-
* Checks if the request contains any input stream.
499+
* Checks if the request contains any input stream or custom writer.
559500
*
560-
* @return true if there's input stream; false otherwise
501+
* @return true if there's input stream or customer writer; false otherwise
561502
*/
562503
public boolean hasInputStream() {
563-
return this.input != null || !this.externalTables.isEmpty();
504+
return this.input != null || this.writer != null || !this.externalTables.isEmpty();
564505
}
565506

566507
/**
@@ -632,9 +573,27 @@ public final BiConsumer<ClickHouseNode, ClickHouseNode> getServerListener() {
632573
* @return input stream
633574
*/
634575
public Optional<ClickHouseInputStream> getInputStream() {
576+
if (this.input == null && this.writer != null) {
577+
final ClickHouseConfig c = getConfig();
578+
final ClickHouseWriter w = this.writer;
579+
this.input = changeProperty(PROP_DATA, this.input,
580+
ClickHouseDeferredValue.of(() -> ClickHouseInputStream.of(c, w)));
581+
}
635582
return input != null ? input.getOptional() : Optional.empty();
636583
}
637584

585+
/**
586+
* Gets custom writer for writing raw request.
587+
*
588+
* @return custom writer
589+
*/
590+
public Optional<ClickHouseWriter> getWriter() {
591+
if (this.writer == null && this.input != null) {
592+
this.writer = changeProperty(PROP_WRITER, this.writer, new PipedWriter(input));
593+
}
594+
return Optional.ofNullable(this.writer);
595+
}
596+
638597
/**
639598
* Gets output stream.
640599
*
@@ -2026,6 +1985,7 @@ public SelfT reset() {
20261985
}
20271986
}
20281987
this.input = changeProperty(PROP_DATA, this.input, null);
1988+
this.writer = changeProperty(PROP_WRITER, this.writer, null);
20291989
this.output = changeProperty(PROP_OUTPUT, this.output, null);
20301990
this.sql = changeProperty(PROP_QUERY, this.sql, null);
20311991
this.preparedQuery = changeProperty(PROP_PREPARED_QUERY, this.preparedQuery, null);
@@ -2065,6 +2025,7 @@ public ClickHouseRequest<SelfT> seal() {
20652025
req.namedParameters.putAll(namedParameters);
20662026

20672027
req.input = input;
2028+
req.writer = writer;
20682029
req.output = output;
20692030
req.queryId = queryId;
20702031
req.sql = sql;

clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -261,7 +261,7 @@ public enum ClickHouseClientOption implements ClickHouseOption {
261261
/**
262262
* Socket timeout in milliseconds.
263263
*/
264-
SOCKET_TIMEOUT("socket_timeout", 30 * 1000, "Socket timeout in milliseconds."),
264+
SOCKET_TIMEOUT("socket_timeout", ClickHouseDataConfig.DEFAULT_TIMEOUT, "Socket timeout in milliseconds."),
265265
/**
266266
* Whether allows for the reuse of local addresses and ports. See
267267
* {@link java.net.StandardSocketOptions#SO_REUSEADDR}.

0 commit comments

Comments
 (0)