Skip to content

Commit 0210d91

Browse files
authored
Merge pull request #2385 from ClickHouse/v2_remove_clickhouse_node
[client-v2] removed usage of ClickHouseNode from main code
2 parents f5a23ca + 0b56036 commit 0210d91

File tree

4 files changed

+114
-37
lines changed

4 files changed

+114
-37
lines changed

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

Lines changed: 49 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,5 @@
11
package com.clickhouse.client.api;
22

3-
import com.clickhouse.client.ClickHouseNode;
43
import com.clickhouse.client.api.command.CommandResponse;
54
import com.clickhouse.client.api.command.CommandSettings;
65
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
@@ -36,9 +35,12 @@
3635
import com.clickhouse.client.api.query.QueryResponse;
3736
import com.clickhouse.client.api.query.QuerySettings;
3837
import com.clickhouse.client.api.query.Records;
38+
import com.clickhouse.client.api.transport.Endpoint;
39+
import com.clickhouse.client.api.transport.HttpEndpoint;
3940
import com.clickhouse.client.config.ClickHouseClientOption;
4041
import com.clickhouse.data.ClickHouseColumn;
4142
import com.clickhouse.data.ClickHouseFormat;
43+
import com.google.common.collect.ImmutableList;
4244
import net.jpountz.lz4.LZ4Factory;
4345
import org.apache.hc.core5.concurrent.DefaultThreadFactory;
4446
import org.apache.hc.core5.http.ClassicHttpResponse;
@@ -78,6 +80,7 @@
7880
import java.util.concurrent.TimeUnit;
7981
import java.util.concurrent.TimeoutException;
8082
import java.util.function.Supplier;
83+
import java.util.stream.Collectors;
8184

8285
import static java.time.temporal.ChronoUnit.MILLIS;
8386
import static java.time.temporal.ChronoUnit.SECONDS;
@@ -116,13 +119,12 @@ public class Client implements AutoCloseable {
116119

117120
private HttpAPIClientHelper httpClientHelper = null;
118121

119-
private final Set<String> endpoints;
122+
private final List<Endpoint> endpoints;
123+
120124
private final Map<String, String> configuration;
121125

122126
private final Map<String, String> readOnlyConfig;
123127

124-
private final List<ClickHouseNode> serverNodes = new ArrayList<>();
125-
126128
// POJO serializer mapping (class -> (schema -> (format -> serializer)))
127129
private final Map<Class<?>, Map<String, Map<String, POJOSerializer>>> serializers;
128130

@@ -154,16 +156,16 @@ private Client(Set<String> endpoints, Map<String,String> configuration, boolean
154156

155157
private Client(Set<String> endpoints, Map<String,String> configuration, boolean useNewImplementation,
156158
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy, Object metricsRegistry) {
157-
this.endpoints = endpoints;
159+
// Simple initialization
158160
this.configuration = configuration;
159161
this.readOnlyConfig = Collections.unmodifiableMap(this.configuration);
160-
this.endpoints.forEach(endpoint -> {
161-
this.serverNodes.add(ClickHouseNode.of(endpoint, this.configuration));
162-
});
163162
this.metricsRegistry = metricsRegistry;
163+
164+
// Serialization
164165
this.serializers = new ConcurrentHashMap<>();
165166
this.deserializers = new ConcurrentHashMap<>();
166167

168+
// Operation Execution
167169
boolean isAsyncEnabled = MapUtils.getFlag(this.configuration, ClientConfigProperties.ASYNC_OPERATIONS.getKey(), false);
168170
if (isAsyncEnabled && sharedOperationExecutor == null) {
169171
this.isSharedOpExecutorOwned = true;
@@ -172,10 +174,29 @@ private Client(Set<String> endpoints, Map<String,String> configuration, boolean
172174
this.isSharedOpExecutorOwned = false;
173175
this.sharedOperationExecutor = sharedOperationExecutor;
174176
}
175-
boolean initSslContext = getEndpoints().stream().anyMatch(s -> s.toLowerCase().contains("https://"));
176-
this.httpClientHelper = new HttpAPIClientHelper(configuration, metricsRegistry, initSslContext);
177+
177178
this.columnToMethodMatchingStrategy = columnToMethodMatchingStrategy;
178179

180+
181+
// Transport
182+
ImmutableList.Builder<Endpoint> tmpEndpoints = ImmutableList.builder();
183+
boolean initSslContext = false;
184+
for (String ep : endpoints) {
185+
try {
186+
HttpEndpoint endpoint = new HttpEndpoint(ep);
187+
if (endpoint.isSecure()) {
188+
initSslContext = true;
189+
}
190+
LOG.debug("Adding endpoint: {}", endpoint);
191+
tmpEndpoints.add(endpoint);
192+
} catch (Exception e) {
193+
throw new ClientException("Failed to add endpoint " + ep, e);
194+
}
195+
}
196+
197+
this.endpoints = tmpEndpoints.build();
198+
this.httpClientHelper = new HttpAPIClientHelper(configuration, metricsRegistry, initSslContext);
199+
179200
String retry = configuration.get(ClientConfigProperties.RETRY_ON_FAILURE.getKey());
180201
this.retries = retry == null ? 0 : Integer.parseInt(retry);
181202
boolean useNativeCompression = !MapUtils.getFlag(configuration, ClientConfigProperties.DISABLE_NATIVE_COMPRESSION.getKey(), false);
@@ -1185,11 +1206,6 @@ private void setDefaults() {
11851206
}
11861207
}
11871208

1188-
private ClickHouseNode getServerNode() {
1189-
// TODO: implement load balancing using existing logic
1190-
return this.serverNodes.get(0);
1191-
}
1192-
11931209
/**
11941210
* Pings the server to check if it is alive
11951211
* @return true if the server is alive, false otherwise
@@ -1374,13 +1390,13 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
13741390
Supplier<InsertResponse> supplier = () -> {
13751391
long startTime = System.nanoTime();
13761392
// Selecting some node
1377-
ClickHouseNode selectedNode = getNextAliveNode();
1393+
Endpoint selectedEndpoint = getNextAliveNode();
13781394

13791395
RuntimeException lastException = null;
13801396
for (int i = 0; i <= maxRetries; i++) {
13811397
// Execute request
13821398
try (ClassicHttpResponse httpResponse =
1383-
httpClientHelper.executeRequest(selectedNode, finalSettings.getAllSettings(), lz4Factory,
1399+
httpClientHelper.executeRequest(selectedEndpoint, finalSettings.getAllSettings(), lz4Factory,
13841400
out -> {
13851401
out.write("INSERT INTO ".getBytes());
13861402
out.write(tableName.getBytes());
@@ -1404,7 +1420,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
14041420
// Check response
14051421
if (httpResponse.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
14061422
LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})", httpResponse.getCode(), System.nanoTime() - startTime);
1407-
selectedNode = getNextAliveNode();
1423+
selectedEndpoint = getNextAliveNode();
14081424
continue;
14091425
}
14101426

@@ -1421,7 +1437,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
14211437
(i + 1), (maxRetries + 1), System.nanoTime() - startTime), e);
14221438
if (httpClientHelper.shouldRetry(e, finalSettings.getAllSettings())) {
14231439
LOG.warn("Retrying.", e);
1424-
selectedNode = getNextAliveNode();
1440+
selectedEndpoint = getNextAliveNode();
14251441
} else {
14261442
throw lastException;
14271443
}
@@ -1591,13 +1607,13 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15911607
responseSupplier = () -> {
15921608
long startTime = System.nanoTime();
15931609
// Selecting some node
1594-
ClickHouseNode selectedNode = getNextAliveNode();
1610+
Endpoint selectedEndpoint = getNextAliveNode();
15951611

15961612
RuntimeException lastException = null;
15971613
for (int i = 0; i <= retries; i++) {
15981614
// Execute request
15991615
try (ClassicHttpResponse httpResponse =
1600-
httpClientHelper.executeRequest(selectedNode, finalSettings.getAllSettings(), lz4Factory,
1616+
httpClientHelper.executeRequest(selectedEndpoint, finalSettings.getAllSettings(), lz4Factory,
16011617
out -> {
16021618
writer.onOutput(out);
16031619
out.close();
@@ -1607,7 +1623,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
16071623
// Check response
16081624
if (httpResponse.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
16091625
LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})", System.nanoTime() - startTime, httpResponse.getCode());
1610-
selectedNode = getNextAliveNode();
1626+
selectedEndpoint = getNextAliveNode();
16111627
continue;
16121628
}
16131629

@@ -1623,7 +1639,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
16231639
(i + 1), (retries + 1), System.nanoTime() - startTime), e);
16241640
if (httpClientHelper.shouldRetry(e, finalSettings.getAllSettings())) {
16251641
LOG.warn("Retrying.", e);
1626-
selectedNode = getNextAliveNode();
1642+
selectedEndpoint = getNextAliveNode();
16271643
} else {
16281644
throw lastException;
16291645
}
@@ -1715,20 +1731,20 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
17151731
responseSupplier = () -> {
17161732
long startTime = System.nanoTime();
17171733
// Selecting some node
1718-
ClickHouseNode selectedNode = getNextAliveNode();
1734+
Endpoint selectedEndpoint = getNextAliveNode();
17191735
RuntimeException lastException = null;
17201736
for (int i = 0; i <= retries; i++) {
17211737
try {
17221738
ClassicHttpResponse httpResponse =
1723-
httpClientHelper.executeRequest(selectedNode, finalSettings.getAllSettings(), lz4Factory, output -> {
1739+
httpClientHelper.executeRequest(selectedEndpoint, finalSettings.getAllSettings(), lz4Factory, output -> {
17241740
output.write(sqlQuery.getBytes(StandardCharsets.UTF_8));
17251741
output.close();
17261742
});
17271743

17281744
// Check response
17291745
if (httpResponse.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
17301746
LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})", System.nanoTime() - startTime, httpResponse.getCode());
1731-
selectedNode = getNextAliveNode();
1747+
selectedEndpoint = getNextAliveNode();
17321748
continue;
17331749
}
17341750

@@ -1753,7 +1769,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
17531769
(i + 1), (retries + 1), System.nanoTime() - startTime), e);
17541770
if (httpClientHelper.shouldRetry(e, finalSettings.getAllSettings())) {
17551771
LOG.warn("Retrying.", e);
1756-
selectedNode = getNextAliveNode();
1772+
selectedEndpoint = getNextAliveNode();
17571773
} else {
17581774
throw lastException;
17591775
}
@@ -1898,8 +1914,9 @@ public <T> List<T> queryAll(String sqlQuery, Class<T> clazz, TableSchema schema)
18981914
* @param allocator - optional supplier to create new instances of the DTO.
18991915
* @throws IllegalArgumentException when class is not registered or no setters found
19001916
* @return List of POJOs filled with data
1901-
* @param <T>
1917+
* @param <T> type of POJO
19021918
*/
1919+
@SuppressWarnings("unchecked")
19031920
public <T> List<T> queryAll(String sqlQuery, Class<T> clazz, TableSchema schema, Supplier<T> allocator) {
19041921
Map<String, POJOSetter> classDeserializers = deserializers.getOrDefault(clazz,
19051922
Collections.emptyMap()).getOrDefault(schema.getTableName() == null?
@@ -2180,9 +2197,10 @@ protected int getOperationTimeout() {
21802197
/**
21812198
* Returns unmodifiable set of endpoints.
21822199
* @return - set of endpoints
2200+
* @deprecated
21832201
*/
21842202
public Set<String> getEndpoints() {
2185-
return Collections.unmodifiableSet(endpoints);
2203+
return endpoints.stream().map(Endpoint::getBaseURL).collect(Collectors.toSet());
21862204
}
21872205

21882206
public String getUser() {
@@ -2236,8 +2254,8 @@ public void updateBearerToken(String bearer) {
22362254
this.configuration.put(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION), "Bearer " + bearer);
22372255
}
22382256

2239-
private ClickHouseNode getNextAliveNode() {
2240-
return serverNodes.get(0);
2257+
private Endpoint getNextAliveNode() {
2258+
return endpoints.get(0);
22412259
}
22422260

22432261
public static final String VALUES_LIST_DELIMITER = ",";

client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,5 @@
11
package com.clickhouse.client.api.internal;
22

3-
import com.clickhouse.client.ClickHouseNode;
43
import com.clickhouse.client.ClickHouseSslContextProvider;
54
import com.clickhouse.client.api.Client;
65
import com.clickhouse.client.api.ClientConfigProperties;
@@ -13,6 +12,7 @@
1312
import com.clickhouse.client.api.data_formats.internal.SerializerUtils;
1413
import com.clickhouse.client.api.enums.ProxyType;
1514
import com.clickhouse.client.api.http.ClickHouseHttpProto;
15+
import com.clickhouse.client.api.transport.Endpoint;
1616
import net.jpountz.lz4.LZ4Factory;
1717
import org.apache.hc.client5.http.ConnectTimeoutException;
1818
import org.apache.hc.client5.http.classic.methods.HttpPost;
@@ -378,8 +378,8 @@ public Exception readError(ClassicHttpResponse httpResponse) {
378378
private static final long POOL_VENT_TIMEOUT = 10000L;
379379
private AtomicLong timeToPoolVent = new AtomicLong(0);
380380

381-
public ClassicHttpResponse executeRequest(ClickHouseNode server, Map<String, Object> requestConfig, LZ4Factory lz4Factory,
382-
IOCallback<OutputStream> writeCallback) throws IOException {
381+
public ClassicHttpResponse executeRequest(Endpoint server, Map<String, Object> requestConfig, LZ4Factory lz4Factory,
382+
IOCallback<OutputStream> writeCallback) throws IOException {
383383
if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) {
384384
timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT);
385385
poolControl.closeExpired();
@@ -390,7 +390,7 @@ public ClassicHttpResponse executeRequest(ClickHouseNode server, Map<String, Obj
390390
}
391391
URI uri;
392392
try {
393-
URIBuilder uriBuilder = new URIBuilder(server.getBaseUri());
393+
URIBuilder uriBuilder = new URIBuilder(server.getBaseURL());
394394
addQueryParams(uriBuilder, chConfiguration, requestConfig);
395395
uri = uriBuilder.normalizeSyntax().build();
396396
} catch (URISyntaxException e) {
@@ -431,10 +431,10 @@ public ClassicHttpResponse executeRequest(ClickHouseNode server, Map<String, Obj
431431
return httpResponse;
432432

433433
} catch (UnknownHostException e) {
434-
LOG.warn("Host '{}' unknown", server.getHost());
434+
LOG.warn("Host '{}' unknown", server.getBaseURL());
435435
throw new ClientException("Unknown host", e);
436436
} catch (ConnectException | NoRouteToHostException e) {
437-
LOG.warn("Failed to connect to '{}': {}", server.getHost(), e.getMessage());
437+
LOG.warn("Failed to connect to '{}': {}", server.getBaseURL(), e.getMessage());
438438
throw new ClientException("Failed to connect", e);
439439
} catch (ConnectionRequestTimeoutException | ServerException | NoHttpResponseException | ClientException | SocketTimeoutException e) {
440440
throw e;
Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,11 @@
1+
package com.clickhouse.client.api.transport;
2+
3+
/**
4+
* Interface defining the behavior of transport endpoint.
5+
* It is transport responsibility to provide suitable implementation.
6+
*/
7+
public interface Endpoint {
8+
9+
String getBaseURL();
10+
11+
}
Lines changed: 48 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,48 @@
1+
package com.clickhouse.client.api.transport;
2+
3+
import java.net.MalformedURLException;
4+
import java.net.URI;
5+
import java.net.URL;
6+
7+
public class HttpEndpoint implements Endpoint {
8+
9+
private final URI uri; // contains complete connection URL + parameters
10+
11+
private final URL url; // only communication part
12+
13+
private final String baseURL;
14+
15+
private final String info;
16+
17+
private final boolean secure;
18+
19+
public HttpEndpoint(String uri) throws MalformedURLException {
20+
this.uri = URI.create(uri);
21+
this.url = this.uri.toURL();
22+
this.baseURL = url.toString();
23+
this.info = baseURL;
24+
this.secure = this.uri.getScheme().equalsIgnoreCase("https");
25+
}
26+
27+
@Override
28+
public String getBaseURL() {
29+
return baseURL;
30+
}
31+
32+
public URL getURL() {
33+
return url;
34+
}
35+
36+
public URI getURI() {
37+
return uri;
38+
}
39+
40+
public boolean isSecure() {
41+
return secure;
42+
}
43+
44+
@Override
45+
public String toString() {
46+
return info;
47+
}
48+
}

0 commit comments

Comments
 (0)