diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index ebc4f8cfa..28cb4235f 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -15,7 +15,6 @@
import com.clickhouse.client.api.http.ClickHouseHttpProto;
import com.clickhouse.client.api.insert.InsertResponse;
import com.clickhouse.client.api.insert.InsertSettings;
-import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
import com.clickhouse.client.api.internal.HttpAPIClientHelper;
import com.clickhouse.client.api.internal.MapUtils;
@@ -80,9 +79,6 @@
import java.util.function.Supplier;
import java.util.stream.Collectors;
-import static java.time.temporal.ChronoUnit.MILLIS;
-import static java.time.temporal.ChronoUnit.SECONDS;
-
/**
*
Client is the starting point for all interactions with ClickHouse.
*
@@ -141,12 +137,12 @@ public class Client implements AutoCloseable {
private int retries;
private LZ4Factory lz4Factory = null;
- private Client(Set endpoints, Map configuration, boolean useNewImplementation,
+ private Client(Set endpoints, Map configuration,
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy) {
- this(endpoints, configuration, useNewImplementation, sharedOperationExecutor, columnToMethodMatchingStrategy, null);
+ this(endpoints, configuration, sharedOperationExecutor, columnToMethodMatchingStrategy, null);
}
- private Client(Set endpoints, Map configuration, boolean useNewImplementation,
+ private Client(Set endpoints, Map configuration,
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy, Object metricsRegistry) {
// Simple initialization
this.configuration = configuration;
@@ -256,14 +252,22 @@ public static class Builder {
// Read-only configuration
private Map configuration;
- private boolean useNewImplementation = true;
private ExecutorService sharedOperationExecutor = null;
private ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy;
private Object metricRegistry = null;
public Builder() {
this.endpoints = new HashSet<>();
- this.configuration = new HashMap();
+ this.configuration = new HashMap<>();
+
+ for (ClientConfigProperties p : ClientConfigProperties.values()) {
+ if (p.getDefaultValue() != null) {
+ this.configuration.put(p.getKey(), p.getDefaultValue());
+ }
+ }
+
+ allowBinaryReaderToReuseBuffers(false);
+ columnToMethodMatchingStrategy = DefaultColumnToMethodMatchingStrategy.INSTANCE;
}
/**
@@ -510,7 +514,7 @@ public Builder setSocketRcvbuf(long size) {
* @param size - socket send buffer size in bytes
*/
public Builder setSocketSndbuf(long size) {
- this.configuration.put(ClientConfigProperties.SOCKET_RCVBUF_OPT.getKey(), String.valueOf(size));
+ this.configuration.put(ClientConfigProperties.SOCKET_SNDBUF_OPT.getKey(), String.valueOf(size));
return this;
}
@@ -656,25 +660,12 @@ public Builder setExecutionTimeout(long timeout, ChronoUnit timeUnit) {
return this;
}
- /**
- * Switches to new implementation of the client. Default is true.
- * Throws exception if {@code useNewImplementation == false}
- * @deprecated
- */
- public Builder useNewImplementation(boolean useNewImplementation) {
- if (!useNewImplementation) {
- throw new ClientException("switch between new and old version is remove because old version is deprecated.");
- }
- return this;
- }
-
public Builder setHttpCookiesEnabled(boolean enabled) {
//TODO: extract to settings string constants
this.configuration.put("client.http.cookies_enabled", String.valueOf(enabled));
return this;
}
-
/**
* Defines path to the trust store file. It cannot be combined with
* certificates. Either trust store or certificates should be used.
@@ -1013,8 +1004,6 @@ public Builder setServerVersion(String serverVersion) {
}
public Client build() {
- setDefaults();
-
// check if endpoint are empty. so can not initiate client
if (this.endpoints.isEmpty()) {
throw new IllegalArgumentException("At least one endpoint is required");
@@ -1070,128 +1059,9 @@ public Client build() {
throw new IllegalArgumentException("Nor server timezone nor specific timezone is set");
}
- return new Client(this.endpoints, this.configuration, this.useNewImplementation, this.sharedOperationExecutor,
+ return new Client(this.endpoints, this.configuration, this.sharedOperationExecutor,
this.columnToMethodMatchingStrategy, this.metricRegistry);
}
-
-
- private static final int DEFAULT_NETWORK_BUFFER_SIZE = 300_000;
-
- /**
- * Default size for a buffers used in networking.
- */
- public static final int DEFAULT_BUFFER_SIZE = 8192;
- public static final int DEFAULT_SOCKET_BUFFER_SIZE = 804800;
-
- private void setDefaults() {
-
- // set default database name if not specified
- if (!configuration.containsKey(ClientConfigProperties.DATABASE.getKey())) {
- setDefaultDatabase((String) "default");
- }
-
- if (!configuration.containsKey(ClientConfigProperties.MAX_EXECUTION_TIME.getKey())) {
- setExecutionTimeout(0, MILLIS);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.MAX_THREADS_PER_CLIENT.getKey())) {
- configuration.put(ClientConfigProperties.MAX_THREADS_PER_CLIENT.getKey(),
- String.valueOf(0));
- }
-
- if (!configuration.containsKey("compression.lz4.uncompressed_buffer_size")) {
- setLZ4UncompressedBufferSize(ClickHouseLZ4OutputStream.UNCOMPRESSED_BUFF_SIZE);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.DISABLE_NATIVE_COMPRESSION.getKey())) {
- disableNativeCompression(false);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey())) {
- useServerTimeZone(true);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.SERVER_TIMEZONE.getKey())) {
- setServerTimeZone("UTC");
- }
-
- if (!configuration.containsKey(ClientConfigProperties.ASYNC_OPERATIONS.getKey())) {
- useAsyncRequests(false);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.getKey())) {
- setMaxConnections(10);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.CONNECTION_REQUEST_TIMEOUT.getKey())) {
- setConnectionRequestTimeout(10, SECONDS);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getKey())) {
- setConnectionReuseStrategy(ConnectionReuseStrategy.FIFO);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.CONNECTION_POOL_ENABLED.getKey())) {
- enableConnectionPool(true);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.CONNECTION_TTL.getKey())) {
- setConnectionTTL(-1, MILLIS);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getKey())) {
- retryOnFailures(ClientFaultCause.NoHttpResponse, ClientFaultCause.ConnectTimeout,
- ClientFaultCause.ConnectionRequestTimeout);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey())) {
- setClientNetworkBufferSize(DEFAULT_NETWORK_BUFFER_SIZE);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.RETRY_ON_FAILURE.getKey())) {
- setMaxRetries(3);
- }
-
- if (!configuration.containsKey("client_allow_binary_reader_to_reuse_buffers")) {
- allowBinaryReaderToReuseBuffers(false);
- }
-
- if (columnToMethodMatchingStrategy == null) {
- columnToMethodMatchingStrategy = DefaultColumnToMethodMatchingStrategy.INSTANCE;
- }
-
- if (!configuration.containsKey(ClientConfigProperties.HTTP_USE_BASIC_AUTH.getKey())) {
- useHTTPBasicAuth(true);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey())) {
- compressClientRequest(false);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getKey())) {
- compressServerResponse(true);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.USE_HTTP_COMPRESSION.getKey())) {
- useHttpCompression(false);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.APP_COMPRESSED_DATA.getKey())) {
- appCompressedData(false);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getKey())) {
- setSocketTimeout(0, SECONDS);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.SOCKET_RCVBUF_OPT.getKey())) {
- setSocketRcvbuf(DEFAULT_SOCKET_BUFFER_SIZE);
- }
-
- if (!configuration.containsKey(ClientConfigProperties.SOCKET_SNDBUF_OPT.getKey())) {
- setSocketSndbuf(DEFAULT_SOCKET_BUFFER_SIZE);
- }
- }
}
/**
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java
index 0327e1e80..06e0a9153 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java
@@ -1,5 +1,7 @@
package com.clickhouse.client.api;
+import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
+
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@@ -16,11 +18,11 @@ public enum ClientConfigProperties {
SETTING_LOG_COMMENT(serverSetting("log_comment")),
- HTTP_USE_BASIC_AUTH("http_use_basic_auth"),
+ HTTP_USE_BASIC_AUTH("http_use_basic_auth", "true"),
USER("user", "default"),
- PASSWORD("password", ""),
+ PASSWORD("password"),
/**
* Maximum number of active connection in internal connection pool.
@@ -32,27 +34,27 @@ public enum ClientConfigProperties {
*/
HTTP_KEEP_ALIVE_TIMEOUT("http_keep_alive_timeout"),
- USE_SERVER_TIMEZONE("use_server_time_zone"),
+ USE_SERVER_TIMEZONE("use_server_time_zone", "true"),
USE_TIMEZONE("use_time_zone"),
SERVER_VERSION("server_version"),
- SERVER_TIMEZONE("server_time_zone"),
+ SERVER_TIMEZONE("server_time_zone", "UTC"),
- ASYNC_OPERATIONS("async"),
+ ASYNC_OPERATIONS("async", "false"),
- CONNECTION_TTL("connection_ttl"),
+ CONNECTION_TTL("connection_ttl", "-1"),
CONNECTION_TIMEOUT("connection_timeout"),
- CONNECTION_REUSE_STRATEGY("connection_reuse_strategy"),
+ CONNECTION_REUSE_STRATEGY("connection_reuse_strategy", String.valueOf(ConnectionReuseStrategy.FIFO)),
- SOCKET_OPERATION_TIMEOUT("socket_timeout"),
+ SOCKET_OPERATION_TIMEOUT("socket_timeout", "0"),
- SOCKET_RCVBUF_OPT("socket_rcvbuf"),
+ SOCKET_RCVBUF_OPT("socket_rcvbuf", "8196"),
- SOCKET_SNDBUF_OPT("socket_sndbuf"),
+ SOCKET_SNDBUF_OPT("socket_sndbuf", "8196"),
SOCKET_REUSEADDR_OPT("socket_reuseaddr"),
@@ -64,13 +66,13 @@ public enum ClientConfigProperties {
DATABASE("database", "default"),
- COMPRESS_SERVER_RESPONSE("compress"), // actually a server setting, but has client effect too
+ COMPRESS_SERVER_RESPONSE("compress", "true"), // actually a server setting, but has client effect too
- COMPRESS_CLIENT_REQUEST("decompress"), // actually a server setting, but has client effect too
+ COMPRESS_CLIENT_REQUEST("decompress", "false"), // actually a server setting, but has client effect too
- USE_HTTP_COMPRESSION("client.use_http_compression"),
+ USE_HTTP_COMPRESSION("client.use_http_compression", "false"),
- COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE("compression.lz4.uncompressed_buffer_size"),
+ COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE("compression.lz4.uncompressed_buffer_size", String.valueOf(ClickHouseLZ4OutputStream.UNCOMPRESSED_BUFF_SIZE)),
DISABLE_NATIVE_COMPRESSION("disable_native_compression", "false"),
@@ -84,7 +86,7 @@ public enum ClientConfigProperties {
PROXY_PASSWORD("proxy_password"),
- MAX_EXECUTION_TIME("max_execution_time"),
+ MAX_EXECUTION_TIME("max_execution_time", "0"),
SSL_TRUST_STORE("trust_store"),
@@ -100,23 +102,25 @@ public enum ClientConfigProperties {
SSL_CERTIFICATE("sslcert"),
- RETRY_ON_FAILURE("retry"),
+ RETRY_ON_FAILURE("retry", "3"),
INPUT_OUTPUT_FORMAT("format"),
- MAX_THREADS_PER_CLIENT("max_threads_per_client"),
+ MAX_THREADS_PER_CLIENT("max_threads_per_client", "0"),
QUERY_ID("query_id"), // actually a server setting, but has client effect too
- CLIENT_NETWORK_BUFFER_SIZE("client_network_buffer_size", String.valueOf(Client.Builder.DEFAULT_BUFFER_SIZE)),
+ CLIENT_NETWORK_BUFFER_SIZE("client_network_buffer_size", "300000"),
ACCESS_TOKEN("access_token"), SSL_AUTH("ssl_authentication"),
- CONNECTION_POOL_ENABLED("connection_pool_enabled"),
+ CONNECTION_POOL_ENABLED("connection_pool_enabled", "true"),
- CONNECTION_REQUEST_TIMEOUT("connection_request_timeout"),
+ CONNECTION_REQUEST_TIMEOUT("connection_request_timeout", "10000"),
- CLIENT_RETRY_ON_FAILURE("client_retry_on_failures"),
+ CLIENT_RETRY_ON_FAILURE("client_retry_on_failures",
+ String.join(",", ClientFaultCause.NoHttpResponse.name(), ClientFaultCause.ConnectTimeout.name(),
+ ClientFaultCause.ConnectionRequestTimeout.name())),
CLIENT_NAME("client_name"),
@@ -130,18 +134,19 @@ public enum ClientConfigProperties {
/**
* Indicates that data provided for write operation is compressed by application.
*/
- APP_COMPRESSED_DATA("app_compressed_data"),
+ APP_COMPRESSED_DATA("app_compressed_data", "false"),
+
/**
- *
+ * Name of the group under which client metrics appear
*/
METRICS_GROUP_NAME("metrics_name"),
;
- private String key;
+ private final String key;
- private String defaultValue;
+ private final String defaultValue;
- private List choices;
+ private final List choices;
ClientConfigProperties(String key) {
diff --git a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java
index 13fd0026f..3499f1dc1 100644
--- a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java
@@ -3,7 +3,11 @@
import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.ClientException;
+import com.clickhouse.client.api.ClientFaultCause;
+import com.clickhouse.client.api.ConnectionReuseStrategy;
import com.clickhouse.client.api.enums.Protocol;
+import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
+import com.clickhouse.client.api.metadata.DefaultColumnToMethodMatchingStrategy;
import com.clickhouse.client.api.query.GenericRecord;
import com.clickhouse.client.api.query.QueryResponse;
import com.clickhouse.client.api.query.QuerySettings;
@@ -24,6 +28,9 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import static java.time.temporal.ChronoUnit.MILLIS;
+import static java.time.temporal.ChronoUnit.SECONDS;
+
public class ClientTests extends BaseIntegrationTest {
private static final Logger LOGGER = LoggerFactory.getLogger(ClientTests.class);
@@ -188,6 +195,121 @@ public void testDisableNative() {
}
}
+ @Test(groups = {"integration"})
+ public void testDefaultSettings() {
+ try (Client client = new Client.Builder().setUsername("default").setPassword("secret")
+ .addEndpoint("http://localhost:8123").build()) {
+ Map config = client.getConfiguration();
+ for (ClientConfigProperties p : ClientConfigProperties.values()) {
+ if (p.getDefaultValue() != null) {
+ Assert.assertTrue(config.containsKey(p.getKey()), "Default value should be set for " + p.getKey());
+ Assert.assertEquals(config.get(p.getKey()), p.getDefaultValue(), "Default value doesn't match");
+ }
+ }
+ Assert.assertEquals(config.size(), 27); // to check everything is set. Increment when new added.
+ }
+
+ try (Client client = new Client.Builder()
+ .setUsername("default")
+ .setPassword("secret")
+ .addEndpoint("http://localhost:8123")
+ .setDefaultDatabase("mydb")
+ .setExecutionTimeout(10, MILLIS)
+ .setLZ4UncompressedBufferSize(300_000)
+ .disableNativeCompression(true)
+ .useServerTimeZone(false)
+ .setServerTimeZone("America/Los_Angeles")
+ .useTimeZone("America/Los_Angeles")
+ .useAsyncRequests(true)
+ .setMaxConnections(330)
+ .setConnectionRequestTimeout(20, SECONDS)
+ .setConnectionReuseStrategy(ConnectionReuseStrategy.LIFO)
+ .enableConnectionPool(false)
+ .setConnectionTTL(30, SECONDS)
+ .retryOnFailures(ClientFaultCause.NoHttpResponse)
+ .setClientNetworkBufferSize(500_000)
+ .setMaxRetries(10)
+ .useHTTPBasicAuth(false)
+ .compressClientRequest(true)
+ .compressServerResponse(false)
+ .useHttpCompression(true)
+ .appCompressedData(true)
+ .setSocketTimeout(20, SECONDS)
+ .setSocketRcvbuf(100000)
+ .setSocketSndbuf(100000)
+ .build()) {
+ Map config = client.getConfiguration();
+ Assert.assertEquals(config.size(), 28); // to check everything is set. Increment when new added.
+ Assert.assertEquals(config.get(ClientConfigProperties.DATABASE.getKey()), "mydb");
+ Assert.assertEquals(config.get(ClientConfigProperties.MAX_EXECUTION_TIME.getKey()), "10");
+ Assert.assertEquals(config.get(ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getKey()), "300000");
+ Assert.assertEquals(config.get(ClientConfigProperties.DISABLE_NATIVE_COMPRESSION.getKey()), "true");
+ Assert.assertEquals(config.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey()), "false");
+ Assert.assertEquals(config.get(ClientConfigProperties.SERVER_TIMEZONE.getKey()), "America/Los_Angeles");
+ Assert.assertEquals(config.get(ClientConfigProperties.ASYNC_OPERATIONS.getKey()), "true");
+ Assert.assertEquals(config.get(ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.getKey()), "330");
+ Assert.assertEquals(config.get(ClientConfigProperties.CONNECTION_REQUEST_TIMEOUT.getKey()), "20000");
+ Assert.assertEquals(config.get(ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getKey()), "LIFO");
+ Assert.assertEquals(config.get(ClientConfigProperties.CONNECTION_POOL_ENABLED.getKey()), "false");
+ Assert.assertEquals(config.get(ClientConfigProperties.CONNECTION_TTL.getKey()), "30000");
+ Assert.assertEquals(config.get(ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getKey()), "NoHttpResponse");
+ Assert.assertEquals(config.get(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey()), "500000");
+ Assert.assertEquals(config.get(ClientConfigProperties.RETRY_ON_FAILURE.getKey()), "10");
+ Assert.assertEquals(config.get(ClientConfigProperties.HTTP_USE_BASIC_AUTH.getKey()), "false");
+ Assert.assertEquals(config.get(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey()), "true");
+ Assert.assertEquals(config.get(ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getKey()), "false");
+ Assert.assertEquals(config.get(ClientConfigProperties.USE_HTTP_COMPRESSION.getKey()), "true");
+ Assert.assertEquals(config.get(ClientConfigProperties.APP_COMPRESSED_DATA.getKey()), "true");
+ Assert.assertEquals(config.get(ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getKey()), "20000");
+ Assert.assertEquals(config.get(ClientConfigProperties.SOCKET_RCVBUF_OPT.getKey()), "100000");
+ Assert.assertEquals(config.get(ClientConfigProperties.SOCKET_SNDBUF_OPT.getKey()), "100000");
+ }
+ }
+
+ @Test(groups = {"integration"})
+ public void testWithOldDefaults() {
+ try (Client client = new Client.Builder()
+ .setUsername("default")
+ .setPassword("seceret")
+ .addEndpoint("http://localhost:8123")
+ .setDefaultDatabase("default")
+ .setExecutionTimeout(0, MILLIS)
+ .setLZ4UncompressedBufferSize(ClickHouseLZ4OutputStream.UNCOMPRESSED_BUFF_SIZE)
+ .disableNativeCompression(false)
+ .useServerTimeZone(true)
+ .setServerTimeZone("UTC")
+ .useAsyncRequests(false)
+ .setMaxConnections(10)
+ .setConnectionRequestTimeout(10, SECONDS)
+ .setConnectionReuseStrategy(ConnectionReuseStrategy.FIFO)
+ .enableConnectionPool(true)
+ .setConnectionTTL(-1, MILLIS)
+ .retryOnFailures(ClientFaultCause.NoHttpResponse, ClientFaultCause.ConnectTimeout,
+ ClientFaultCause.ConnectionRequestTimeout)
+ .setClientNetworkBufferSize(300_000)
+ .setMaxRetries(3)
+ .allowBinaryReaderToReuseBuffers(false)
+ .columnToMethodMatchingStrategy(DefaultColumnToMethodMatchingStrategy.INSTANCE)
+ .useHTTPBasicAuth(true)
+ .compressClientRequest(false)
+ .compressServerResponse(true)
+ .useHttpCompression(false)
+ .appCompressedData(false)
+ .setSocketTimeout(0, SECONDS)
+ .setSocketRcvbuf(8196)
+ .setSocketSndbuf(8196)
+ .build()) {
+ Map config = client.getConfiguration();
+ for (ClientConfigProperties p : ClientConfigProperties.values()) {
+ if (p.getDefaultValue() != null) {
+ Assert.assertTrue(config.containsKey(p.getKey()), "Default value should be set for " + p.getKey());
+ Assert.assertEquals(config.get(p.getKey()), p.getDefaultValue(), "Default value doesn't match");
+ }
+ }
+ Assert.assertEquals(config.size(), 27); // to check everything is set. Increment when new added.
+ }
+ }
+
protected Client.Builder newClient() {
ClickHouseNode node = getServer(ClickHouseProtocol.HTTP);
boolean isSecure = isCloud();
diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java
index 1a31d325e..23b29b7c1 100644
--- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java
@@ -90,7 +90,6 @@ public void testConnectionTTL(Long connectionTtl, Long keepAlive, int openSocket
.addEndpoint(server.getBaseUri())
.setUsername("default")
.setPassword(getPassword())
- .useNewImplementation(true)
.addProxy(ProxyType.HTTP, "localhost", proxyPort);
if (connectionTtl != null) {
clientBuilder.setConnectionTTL(connectionTtl, ChronoUnit.MILLIS);
@@ -180,7 +179,6 @@ public void testConnectionRequestTimeout() {
.setUsername("default")
.setPassword(getPassword())
.retryOnFailures(ClientFaultCause.None)
- .useNewImplementation(true)
.setMaxConnections(1)
.setOption(ClickHouseClientOption.ASYNC.getKey(), "true")
.setSocketTimeout(10000, ChronoUnit.MILLIS)
@@ -215,7 +213,6 @@ public void testConnectionReuseStrategy() {
.addEndpoint(server.getBaseUri())
.setUsername("default")
.setPassword(getPassword())
- .useNewImplementation(true)
.setConnectionReuseStrategy(ConnectionReuseStrategy.LIFO)
.build()) {
@@ -286,7 +283,6 @@ public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, Thro
.addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false)
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
- .useNewImplementation(true) // because of the internal differences
.compressClientRequest(false)
.setMaxRetries(maxRetries)
.build();
@@ -438,7 +434,6 @@ public void testErrorWithSuccessfulResponse() {
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
.compressServerResponse(false)
- .useNewImplementation(true)
.build()) {
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
.willReturn(WireMock.aResponse()
@@ -537,7 +532,6 @@ public void testAdditionalHeaders() {
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
- .useNewImplementation(true)
.httpHeader("X-ClickHouse-Test", "default_value")
.httpHeader("X-ClickHouse-Test-2", Arrays.asList("default_value1", "default_value2"))
.httpHeader("X-ClickHouse-Test-3", Arrays.asList("default_value1", "default_value2"))
@@ -581,7 +575,6 @@ public void testServerSettings() {
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
- .useNewImplementation(true)
.serverSetting("max_threads", "10")
.serverSetting("async_insert", "1")
.serverSetting("roles", Arrays.asList("role1", "role2"))
@@ -779,7 +772,6 @@ public void testErrorWithSendProgressHeaders() throws Exception {
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false)
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
- .useNewImplementation(true)
.build()) {
try (CommandResponse resp = client.execute("DROP TABLE IF EXISTS test_omm_table").get()) {
@@ -1051,7 +1043,6 @@ public void testWithDefaultTimeouts() {
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", proxyPort, false)
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
- .useNewImplementation(true)
.build()) {
int startTime = (int) System.currentTimeMillis();
try {
@@ -1126,7 +1117,6 @@ protected Client.Builder newClient() {
.setPassword(ClickHouseServerForTest.getPassword())
.compressClientRequest(false)
.setDefaultDatabase(ClickHouseServerForTest.getDatabase())
- .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1")
- .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"));
+ .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1");
}
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java b/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java
index 4f024529e..af2a510c3 100644
--- a/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java
@@ -207,8 +207,6 @@ private Client.Builder clientBuilder(int proxyPort, boolean onlyNewImplementatio
.addEndpoint(Protocol.HTTP, "clickhouse", 8123, false)
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
- .useNewImplementation(onlyNewImplementation ? onlyNewImplementation :
- System.getProperty("client.tests.useNewImplementation", "true").equals("true"))
.addProxy(ProxyType.HTTP, "localhost", proxyPort);
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/command/CommandTests.java b/client-v2/src/test/java/com/clickhouse/client/command/CommandTests.java
index 6a631005d..c09842f71 100644
--- a/client-v2/src/test/java/com/clickhouse/client/command/CommandTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/command/CommandTests.java
@@ -59,7 +59,6 @@ protected Client.Builder newClient() {
.setPassword(ClickHouseServerForTest.getPassword())
.compressClientRequest(false)
.setDefaultDatabase(ClickHouseServerForTest.getDatabase())
- .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1")
- .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"));
+ .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1");
}
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/metadata/MetadataTests.java b/client-v2/src/test/java/com/clickhouse/client/metadata/MetadataTests.java
index 8ea740d13..86722f8fd 100644
--- a/client-v2/src/test/java/com/clickhouse/client/metadata/MetadataTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/metadata/MetadataTests.java
@@ -114,7 +114,6 @@ protected Client.Builder newClient() {
.setUsername("default")
.setPassword(ClickHouseServerForTest.getPassword())
.setDefaultDatabase(ClickHouseServerForTest.getDatabase())
- .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1")
- .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"));
+ .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1");
}
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java
index b287e8786..908a75f2b 100644
--- a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java
@@ -2045,8 +2045,7 @@ protected Client.Builder newClient() {
.allowBinaryReaderToReuseBuffers(usePreallocatedBuffers)
.setDefaultDatabase(ClickHouseServerForTest.getDatabase())
.serverSetting(ServerSettings.WAIT_ASYNC_INSERT, "1")
- .serverSetting(ServerSettings.ASYNC_INSERT, "0")
- .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"));
+ .serverSetting(ServerSettings.ASYNC_INSERT, "0");
}
@Test(groups = {"integration"})
diff --git a/examples/client-v2/src/main/java/com/clickhouse/examples/client_v2/BigDatasetExamples.java b/examples/client-v2/src/main/java/com/clickhouse/examples/client_v2/BigDatasetExamples.java
index 18445a102..1efcd1194 100644
--- a/examples/client-v2/src/main/java/com/clickhouse/examples/client_v2/BigDatasetExamples.java
+++ b/examples/client-v2/src/main/java/com/clickhouse/examples/client_v2/BigDatasetExamples.java
@@ -40,7 +40,6 @@ void readBigSetOfNumbers(int limit, int iterations, int concurrency) {
.compressServerResponse(false)
.compressClientRequest(false)
.setLZ4UncompressedBufferSize(1048576)
- .useNewImplementation(true)
// when network buffer and socket buffer are the same size - it is less IO calls and more efficient
.setSocketRcvbuf(1_000_000)
.setClientNetworkBufferSize(1_000_000)
diff --git a/examples/demo-kotlin-service/src/main/kotlin/com/clickhouse/examples/service/Database.kt b/examples/demo-kotlin-service/src/main/kotlin/com/clickhouse/examples/service/Database.kt
index a0cb7d036..053e7f375 100644
--- a/examples/demo-kotlin-service/src/main/kotlin/com/clickhouse/examples/service/Database.kt
+++ b/examples/demo-kotlin-service/src/main/kotlin/com/clickhouse/examples/service/Database.kt
@@ -13,7 +13,6 @@ object Database {
.addEndpoint(dbUrl)
.setUsername(dbUser)
.setPassword(dbPassword)
- .useNewImplementation(true)
.build()
}