Skip to content

Commit 8b8415f

Browse files
committed
Fix build failure
1 parent 8d6ce58 commit 8b8415f

File tree

10 files changed

+173
-94
lines changed

10 files changed

+173
-94
lines changed

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

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -75,11 +75,13 @@ public abstract class ClickHouseInputStream extends InputStream {
7575
* @param compressionLevel compression level
7676
* @return non-null wrapped input stream
7777
*/
78-
static ClickHouseInputStream wrap(ClickHouseFile file, InputStream input, int bufferSize, Runnable postCloseAction,
79-
ClickHouseCompression compression, int compressionLevel) {
78+
public static ClickHouseInputStream wrap(ClickHouseFile file, InputStream input, int bufferSize,
79+
Runnable postCloseAction, ClickHouseCompression compression, int compressionLevel) {
8080
final ClickHouseInputStream chInput;
8181
if (compression == null || compression == ClickHouseCompression.NONE) {
82-
chInput = new WrappedInputStream(file, input, bufferSize, postCloseAction);
82+
chInput = input != EmptyInputStream.INSTANCE && input instanceof ClickHouseInputStream
83+
? (ClickHouseInputStream) input
84+
: new WrappedInputStream(file, input, bufferSize, postCloseAction);
8385
} else {
8486
switch (compression) {
8587
case GZIP:
@@ -485,7 +487,7 @@ public static File save(File file, InputStream in, int bufferSize, int timeout,
485487
tmp = File.createTempFile("chc", "data");
486488
tmp.deleteOnExit();
487489
} catch (IOException e) {
488-
throw new IllegalStateException("Failed to create temp file", e);
490+
throw new UncheckedIOException("Failed to create temp file", e);
489491
}
490492
}
491493
CompletableFuture<File> data = CompletableFuture.supplyAsync(() -> {
@@ -509,7 +511,9 @@ public static File save(File file, InputStream in, int bufferSize, int timeout,
509511
} catch (ExecutionException e) {
510512
Throwable cause = e.getCause();
511513
if (cause instanceof UncheckedIOException) {
512-
cause = ((UncheckedIOException) cause).getCause();
514+
throw ((UncheckedIOException) cause);
515+
} else if (cause instanceof IOException) {
516+
throw new UncheckedIOException((IOException) cause);
513517
}
514518
throw new IllegalStateException(cause);
515519
}

clickhouse-grpc-client/pom.xml

Lines changed: 22 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -39,15 +39,15 @@
3939
<groupId>com.google.code.gson</groupId>
4040
<artifactId>gson</artifactId>
4141
</dependency>
42+
<dependency>
43+
<groupId>org.apache.commons</groupId>
44+
<artifactId>commons-compress</artifactId>
45+
</dependency>
4246
<dependency>
4347
<groupId>io.grpc</groupId>
4448
<artifactId>grpc-protobuf</artifactId>
4549
<scope>provided</scope>
4650
</dependency>
47-
<dependency>
48-
<groupId>org.apache.commons</groupId>
49-
<artifactId>commons-compress</artifactId>
50-
</dependency>
5151
<!-- necessary for Java 9+ -->
5252
<dependency>
5353
<groupId>org.apache.tomcat</groupId>
@@ -119,6 +119,10 @@
119119
<pattern>okio</pattern>
120120
<shadedPattern>${shade.base}.okio</shadedPattern>
121121
</relocation>
122+
<relocation>
123+
<pattern>org.apache</pattern>
124+
<shadedPattern>${shade.base}.apache</shadedPattern>
125+
</relocation>
122126
<relocation>
123127
<pattern>io.grpc</pattern>
124128
<shadedPattern>${shade.base}.grpc</shadedPattern>
@@ -153,7 +157,8 @@
153157
<exclude>android/**</exclude>
154158
<exclude>google/**</exclude>
155159
<exclude>javax/**</exclude>
156-
<exclude>org/**</exclude>
160+
<exclude>org/checkerframework/**</exclude>
161+
<exclude>org/codehaus/**</exclude>
157162
<exclude>**/module-info.class</exclude>
158163
<exclude>META-INF/MANIFEST.MF</exclude>
159164
<exclude>META-INF/maven/**</exclude>
@@ -179,6 +184,10 @@
179184
<pattern>com.google</pattern>
180185
<shadedPattern>${shade.base}.google</shadedPattern>
181186
</relocation>
187+
<relocation>
188+
<pattern>org.apache</pattern>
189+
<shadedPattern>${shade.base}.apache</shadedPattern>
190+
</relocation>
182191
<relocation>
183192
<pattern>io.grpc</pattern>
184193
<shadedPattern>${shade.base}.grpc</shadedPattern>
@@ -211,7 +220,8 @@
211220
<exclude>io/grpc/okhttp/**</exclude>
212221
<exclude>javax/**</exclude>
213222
<exclude>okio/**</exclude>
214-
<exclude>org/**</exclude>
223+
<exclude>org/checkerframework/**</exclude>
224+
<exclude>org/codehaus/**</exclude>
215225
<exclude>**/module-info.class</exclude>
216226
<exclude>META-INF/MANIFEST.MF</exclude>
217227
<exclude>META-INF/maven/**</exclude>
@@ -245,6 +255,10 @@
245255
<pattern>okio</pattern>
246256
<shadedPattern>${shade.base}.okio</shadedPattern>
247257
</relocation>
258+
<relocation>
259+
<pattern>org.apache</pattern>
260+
<shadedPattern>${shade.base}.apache</shadedPattern>
261+
</relocation>
248262
<relocation>
249263
<pattern>io.grpc</pattern>
250264
<shadedPattern>${shade.base}.grpc</shadedPattern>
@@ -275,7 +289,8 @@
275289
<exclude>google/**</exclude>
276290
<exclude>io/grpc/netty/**</exclude>
277291
<exclude>javax/**</exclude>
278-
<exclude>org/**</exclude>
292+
<exclude>org/checkerframework/**</exclude>
293+
<exclude>org/codehaus/**</exclude>
279294
<exclude>**/module-info.class</exclude>
280295
<exclude>META-INF/MANIFEST.MF</exclude>
281296
<exclude>META-INF/maven/**</exclude>

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpClient.java

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99
import java.util.concurrent.CompletionException;
1010

1111
import com.clickhouse.client.AbstractClient;
12+
import com.clickhouse.client.ClickHouseConfig;
1213
import com.clickhouse.client.ClickHouseException;
1314
import com.clickhouse.client.ClickHouseNode;
1415
import com.clickhouse.client.ClickHouseProtocol;
@@ -29,7 +30,7 @@ public class ClickHouseHttpClient extends AbstractClient<ClickHouseHttpConnectio
2930
protected boolean checkConnection(ClickHouseHttpConnection connection, ClickHouseNode requestServer,
3031
ClickHouseNode currentServer, ClickHouseRequest<?> request) {
3132
// return false to suggest creating a new connection
32-
return connection != null && connection.isReusable() && requestServer.equals(currentServer);
33+
return connection != null && connection.isReusable() && requestServer.isSameEndpoint(currentServer);
3334
}
3435

3536
@Override
@@ -98,8 +99,18 @@ protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws Cli
9899
}
99100

100101
log.debug("Query: %s", sql);
101-
ClickHouseHttpResponse httpResponse = conn.post(sql, sealedRequest.getInputStream().orElse(null),
102-
sealedRequest.getExternalTables(), null);
102+
ClickHouseConfig config = sealedRequest.getConfig();
103+
final ClickHouseHttpResponse httpResponse;
104+
if (conn.isReusable()) {
105+
ClickHouseNode server = sealedRequest.getServer();
106+
httpResponse = conn.post(sql, sealedRequest.getInputStream().orElse(null),
107+
sealedRequest.getExternalTables(),
108+
ClickHouseHttpConnection.buildUrl(server.getBaseUri(), sealedRequest),
109+
ClickHouseHttpConnection.createDefaultHeaders(config, server), config);
110+
} else {
111+
httpResponse = conn.post(sql, sealedRequest.getInputStream().orElse(null),
112+
sealedRequest.getExternalTables(), null, null, config);
113+
}
103114
return ClickHouseStreamResponse.of(httpResponse.getConfig(sealedRequest), httpResponse.getInputStream(),
104115
sealedRequest.getSettings(), null, httpResponse.summary);
105116
}

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java

Lines changed: 38 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,11 @@
2626
import com.clickhouse.client.http.config.ClickHouseHttpOption;
2727

2828
public abstract class ClickHouseHttpConnection implements AutoCloseable {
29+
private static StringBuilder appendQueryParameter(StringBuilder builder, String key, String value) {
30+
return builder.append(urlEncode(key, StandardCharsets.UTF_8)).append('=')
31+
.append(urlEncode(value, StandardCharsets.UTF_8)).append('&');
32+
}
33+
2934
static String urlEncode(String str, Charset charset) {
3035
if (charset == null) {
3136
charset = StandardCharsets.UTF_8;
@@ -39,11 +44,6 @@ static String urlEncode(String str, Charset charset) {
3944
}
4045
}
4146

42-
private static StringBuilder appendQueryParameter(StringBuilder builder, String key, String value) {
43-
return builder.append(urlEncode(key, StandardCharsets.UTF_8)).append('=')
44-
.append(urlEncode(value, StandardCharsets.UTF_8)).append('&');
45-
}
46-
4747
static String buildQueryParams(ClickHouseRequest<?> request) {
4848
if (request == null) {
4949
return "";
@@ -146,26 +146,7 @@ static String buildUrl(String baseUrl, ClickHouseRequest<?> request) {
146146
return builder.toString();
147147
}
148148

149-
protected final ClickHouseConfig config;
150-
protected final ClickHouseNode server;
151-
protected final Map<String, String> defaultHeaders;
152-
153-
protected final ClickHouseOutputStream output;
154-
155-
protected final String url;
156-
157-
protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> request) {
158-
if (server == null || request == null) {
159-
throw new IllegalArgumentException("Non-null server and request are required");
160-
}
161-
162-
this.config = request.getConfig();
163-
this.server = server;
164-
165-
this.output = request.getOutputStream().orElse(null);
166-
167-
this.url = buildUrl(server.getBaseUri(), request);
168-
149+
protected static Map<String, String> createDefaultHeaders(ClickHouseConfig config, ClickHouseNode server) {
169150
Map<String, String> map = new LinkedHashMap<>();
170151
// add customer headers
171152
map.putAll(ClickHouseUtils.getKeyValuePairs((String) config.getOption(ClickHouseHttpOption.CUSTOM_HEADERS)));
@@ -201,8 +182,25 @@ protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> r
201182
&& config.getRequestCompressAlgorithm() != ClickHouseCompression.LZ4) {
202183
map.put("Content-Encoding", config.getRequestCompressAlgorithm().encoding());
203184
}
185+
return map;
186+
}
204187

205-
this.defaultHeaders = Collections.unmodifiableMap(map);
188+
protected final ClickHouseConfig config;
189+
protected final ClickHouseNode server;
190+
protected final ClickHouseOutputStream output;
191+
protected final String url;
192+
protected final Map<String, String> defaultHeaders;
193+
194+
protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> request) {
195+
if (server == null || request == null) {
196+
throw new IllegalArgumentException("Non-null server and request are required");
197+
}
198+
199+
this.config = request.getConfig();
200+
this.server = server;
201+
this.output = request.getOutputStream().orElse(null);
202+
this.url = buildUrl(server.getBaseUri(), request);
203+
this.defaultHeaders = Collections.unmodifiableMap(createDefaultHeaders(config, server));
206204
}
207205

208206
protected void closeQuietly() {
@@ -231,11 +229,13 @@ protected String getBaseUrl() {
231229
* Creates a merged map.
232230
*
233231
* @param requestHeaders request headers
234-
* @return
232+
* @return non-null merged headers
235233
*/
236234
protected Map<String, String> mergeHeaders(Map<String, String> requestHeaders) {
237235
if (requestHeaders == null || requestHeaders.isEmpty()) {
238236
return defaultHeaders;
237+
} else if (isReusable()) {
238+
return requestHeaders;
239239
}
240240

241241
Map<String, String> merged = new LinkedHashMap<>();
@@ -256,13 +256,15 @@ protected Map<String, String> mergeHeaders(Map<String, String> requestHeaders) {
256256
* @param query non-blank query
257257
* @param data optionally input stream for batch updating
258258
* @param tables optionally external tables for query
259+
* @param url optionally url
259260
* @param headers optionally request headers
261+
* @param config optionally configuration
260262
* @return response
261263
* @throws IOException when error occured posting request and/or server failed
262264
* to respond
263265
*/
264266
protected abstract ClickHouseHttpResponse post(String query, InputStream data, List<ClickHouseExternalTable> tables,
265-
Map<String, String> headers) throws IOException;
267+
String url, Map<String, String> headers, ClickHouseConfig config) throws IOException;
266268

267269
/**
268270
* Checks whether the connection is reusable or not. This method will be called
@@ -287,36 +289,36 @@ protected boolean isReusable() {
287289
public abstract boolean ping(int timeout);
288290

289291
public ClickHouseHttpResponse update(String query) throws IOException {
290-
return post(query, null, null, null);
292+
return post(query, null, null, null, null, null);
291293
}
292294

293295
public ClickHouseHttpResponse update(String query, Map<String, String> headers) throws IOException {
294-
return post(query, null, null, headers);
296+
return post(query, null, null, null, headers, null);
295297
}
296298

297299
public ClickHouseHttpResponse update(String query, InputStream data) throws IOException {
298-
return post(query, data, null, null);
300+
return post(query, data, null, null, null, null);
299301
}
300302

301303
public ClickHouseHttpResponse update(String query, InputStream data, Map<String, String> headers)
302304
throws IOException {
303-
return post(query, data, null, headers);
305+
return post(query, data, null, null, headers, null);
304306
}
305307

306308
public ClickHouseHttpResponse query(String query) throws IOException {
307-
return post(query, null, null, null);
309+
return post(query, null, null, null, null, null);
308310
}
309311

310312
public ClickHouseHttpResponse query(String query, Map<String, String> headers) throws IOException {
311-
return post(query, null, null, headers);
313+
return post(query, null, null, null, headers, null);
312314
}
313315

314316
public ClickHouseHttpResponse query(String query, List<ClickHouseExternalTable> tables) throws IOException {
315-
return post(query, null, tables, null);
317+
return post(query, null, tables, null, null, null);
316318
}
317319

318320
public ClickHouseHttpResponse query(String query, List<ClickHouseExternalTable> tables, Map<String, String> headers)
319321
throws IOException {
320-
return post(query, null, tables, headers);
322+
return post(query, null, tables, null, headers, null);
321323
}
322324
}

clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -202,7 +202,7 @@ protected boolean isReusable() {
202202

203203
@Override
204204
protected ClickHouseHttpResponse post(String sql, InputStream data, List<ClickHouseExternalTable> tables,
205-
Map<String, String> headers) throws IOException {
205+
String url, Map<String, String> headers, ClickHouseConfig config) throws IOException {
206206
Charset charset = StandardCharsets.US_ASCII;
207207
byte[] boundary = null;
208208
if (tables != null && !tables.isEmpty()) {

0 commit comments

Comments
 (0)