Skip to content

Commit b7b4544

Browse files
authored
fix proxy configuration globally for the client (#1694)
1 parent 4de1232 commit b7b4544

File tree

2 files changed

+83
-38
lines changed

2 files changed

+83
-38
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 14 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
import com.clickhouse.client.api.insert.POJOSerializer;
2020
import com.clickhouse.client.api.insert.SerializerNotFoundException;
2121
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
22+
import com.clickhouse.client.api.internal.ClientV1AdaptorHelper;
2223
import com.clickhouse.client.api.internal.SerializerUtils;
2324
import com.clickhouse.client.api.internal.SettingsConverter;
2425
import com.clickhouse.client.api.internal.TableSchemaParser;
@@ -30,6 +31,7 @@
3031
import com.clickhouse.client.api.query.QuerySettings;
3132
import com.clickhouse.client.api.query.Records;
3233
import com.clickhouse.client.config.ClickHouseClientOption;
34+
import com.clickhouse.config.ClickHouseOption;
3335
import com.clickhouse.data.ClickHouseColumn;
3436
import com.clickhouse.data.ClickHouseDataStreamFactory;
3537
import com.clickhouse.data.ClickHouseFormat;
@@ -42,6 +44,7 @@
4244
import java.io.ByteArrayOutputStream;
4345
import java.io.IOException;
4446
import java.io.InputStream;
47+
import java.io.Serializable;
4548
import java.lang.reflect.InvocationTargetException;
4649
import java.lang.reflect.Method;
4750
import java.net.URL;
@@ -430,10 +433,11 @@ public boolean ping() {
430433
public boolean ping(long timeout) {
431434
ValidationUtils.checkRange(timeout, TimeUnit.SECONDS.toMillis(1), TimeUnit.MINUTES.toMillis(10),
432435
"timeout");
433-
ClickHouseClient clientPing = ClickHouseClient.newInstance(ClickHouseProtocol.HTTP);
434-
return clientPing.ping(getServerNode(), Math.toIntExact(timeout));
435-
}
436436

437+
try (ClickHouseClient client = ClientV1AdaptorHelper.createClient(configuration)) {
438+
return client.ping(getServerNode(), Math.toIntExact(timeout));
439+
}
440+
}
437441

438442
/**
439443
* <p>Registers a POJO class and maps its fields to a table schema</p>
@@ -615,8 +619,9 @@ public CompletableFuture<InsertResponse> insert(String tableName,
615619

616620
CompletableFuture<InsertResponse> responseFuture = new CompletableFuture<>();
617621

618-
try (ClickHouseClient client = createClient()) {
619-
ClickHouseRequest.Mutation request = createMutationRequest(client.write(getServerNode()), tableName, settings).format(format);
622+
try (ClickHouseClient client = ClientV1AdaptorHelper.createClient(configuration)) {
623+
ClickHouseRequest.Mutation request = ClientV1AdaptorHelper
624+
.createMutationRequest(client.write(getServerNode()), tableName, settings, configuration).format(format);
620625
CompletableFuture<ClickHouseResponse> future = null;
621626
try(ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(request.getConfig())) {
622627
future = request.data(stream.getInputStream()).execute();
@@ -708,10 +713,8 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
708713
}
709714
ClientStatisticsHolder clientStats = new ClientStatisticsHolder();
710715
clientStats.start(ClientMetrics.OP_DURATION);
711-
ClickHouseClient client = createClient();
716+
ClickHouseClient client = ClientV1AdaptorHelper.createClient(configuration);
712717
ClickHouseRequest<?> request = client.read(getServerNode());
713-
714-
715718
request.options(SettingsConverter.toRequestOptions(settings.getAllSettings()));
716719
request.settings(SettingsConverter.toRequestSettings(settings.getAllSettings(), queryParams));
717720
request.query(sqlQuery, settings.getQueryId());
@@ -762,10 +765,8 @@ public CompletableFuture<Records> queryRecords(String sqlQuery, QuerySettings se
762765
settings.setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes);
763766
ClientStatisticsHolder clientStats = new ClientStatisticsHolder();
764767
clientStats.start("query");
765-
ClickHouseClient client = createClient();
768+
ClickHouseClient client = ClientV1AdaptorHelper.createClient(configuration);
766769
ClickHouseRequest<?> request = client.read(getServerNode());
767-
768-
769770
request.options(SettingsConverter.toRequestOptions(settings.getAllSettings()));
770771
request.settings(SettingsConverter.toRequestSettings(settings.getAllSettings(), null));
771772
request.query(sqlQuery, settings.getQueryId());
@@ -836,8 +837,8 @@ public TableSchema getTableSchema(String table) {
836837
* @return {@code TableSchema} - Schema of the table
837838
*/
838839
public TableSchema getTableSchema(String table, String database) {
839-
try (ClickHouseClient clientQuery = createClient()) {
840-
ClickHouseRequest request = clientQuery.read(getServerNode());
840+
try (ClickHouseClient clientQuery = ClientV1AdaptorHelper.createClient(configuration)) {
841+
ClickHouseRequest<?> request = clientQuery.read(getServerNode());
841842
// XML - because java has a built-in XML parser. Will consider CSV later.
842843
request.query("DESCRIBE TABLE " + table + " FORMAT " + ClickHouseFormat.TSKV.name());
843844
try {
@@ -848,31 +849,6 @@ public TableSchema getTableSchema(String table, String database) {
848849
}
849850
}
850851

851-
private ClickHouseClient createClient() {
852-
ClickHouseConfig clientConfig = new ClickHouseConfig();
853-
ClickHouseClientBuilder clientV1 = ClickHouseClient.builder()
854-
.config(clientConfig)
855-
.nodeSelector(ClickHouseNodeSelector.of(ClickHouseProtocol.HTTP));
856-
return clientV1.build();
857-
}
858-
859-
private ClickHouseRequest.Mutation createMutationRequest(ClickHouseRequest.Mutation request, String tableName, InsertSettings settings) {
860-
if (settings == null) return request.table(tableName);
861-
862-
if (settings.getQueryId() != null) {//This has to be handled separately
863-
request.table(tableName, settings.getQueryId());
864-
} else {
865-
request.table(tableName);
866-
}
867-
868-
//For each setting, set the value in the request
869-
for (Map.Entry<String, Object> entry : settings.getAllSettings().entrySet()) {
870-
request.set(entry.getKey(), String.valueOf(entry.getValue()));
871-
}
872-
873-
return request;
874-
}
875-
876852
private String startOperation() {
877853
String operationId = UUID.randomUUID().toString();
878854
globalClientStats.put(operationId, new ClientStatisticsHolder());
Lines changed: 69 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,69 @@
1+
package com.clickhouse.client.api.internal;
2+
3+
import com.clickhouse.client.ClickHouseClient;
4+
import com.clickhouse.client.ClickHouseClientBuilder;
5+
import com.clickhouse.client.ClickHouseConfig;
6+
import com.clickhouse.client.ClickHouseNodeSelector;
7+
import com.clickhouse.client.ClickHouseProtocol;
8+
import com.clickhouse.client.ClickHouseRequest;
9+
import com.clickhouse.client.api.insert.InsertSettings;
10+
import com.clickhouse.client.config.ClickHouseClientOption;
11+
import com.clickhouse.client.config.ClickHouseProxyType;
12+
import com.clickhouse.config.ClickHouseOption;
13+
14+
import java.io.Serializable;
15+
import java.util.Collections;
16+
import java.util.HashMap;
17+
import java.util.Map;
18+
19+
public class ClientV1AdaptorHelper {
20+
21+
private static void copyProxySettings(Map<ClickHouseOption, Serializable> target, Map<String, String> config) {
22+
ClickHouseClientOption opt = ClickHouseClientOption.PROXY_HOST;
23+
String value = config.get(opt.getKey());
24+
if (value != null) {
25+
target.put(opt, value);
26+
}
27+
opt = ClickHouseClientOption.PROXY_PORT;
28+
value = config.get(opt.getKey());
29+
if (value != null) {
30+
target.put(opt, Integer.parseInt(value));
31+
}
32+
opt = ClickHouseClientOption.PROXY_TYPE;
33+
value = config.get(opt.getKey());
34+
if (value != null) {
35+
target.put(opt, ClickHouseProxyType.valueOf(value));
36+
}
37+
}
38+
39+
public static ClickHouseClient createClient(Map<String, String> configuration) {
40+
Map<ClickHouseOption, Serializable> config = new HashMap<>();
41+
copyProxySettings(config, configuration);
42+
43+
ClickHouseConfig clientConfig = new ClickHouseConfig(config);
44+
45+
ClickHouseClientBuilder clientV1 = ClickHouseClient.builder()
46+
.config(clientConfig)
47+
.nodeSelector(ClickHouseNodeSelector.of(ClickHouseProtocol.HTTP));
48+
return clientV1.build();
49+
}
50+
51+
public static ClickHouseRequest.Mutation createMutationRequest(ClickHouseRequest.Mutation request,
52+
String tableName,
53+
InsertSettings settings,
54+
Map<String, String> configuration) {
55+
if (settings.getQueryId() != null) {//This has to be handled separately
56+
request.table(tableName, settings.getQueryId());
57+
} else {
58+
request.table(tableName);
59+
}
60+
61+
Map<String, Object> opSettings = settings == null ? Collections.emptyMap() : settings.getAllSettings();
62+
//For each setting, set the value in the request
63+
for (Map.Entry<String, Object> entry : opSettings.entrySet()) {
64+
request.set(entry.getKey(), String.valueOf(entry.getValue()));
65+
}
66+
67+
return request;
68+
}
69+
}

0 commit comments

Comments
 (0)