From f0a88f2700b230ad617d3ad835f0c84a4f43852a Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 11 Dec 2025 19:38:34 -0800 Subject: [PATCH 01/11] preserving path part from url when addEndpoint(string) used --- .../com/clickhouse/client/api/Client.java | 24 +++++-- .../clickhouse/client/HttpTransportTests.java | 65 +++++++++++++++++++ 2 files changed, 83 insertions(+), 6 deletions(-) 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 f7cc187d4..7efbab164 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 @@ -288,21 +288,33 @@ public Builder() { * * - * @param endpoint - URL formatted string with protocol, host and port. + * @param endpoint - URL formatted string with protocol, host, port, and optional path. */ public Builder addEndpoint(String endpoint) { try { URL endpointURL = new URL(endpoint); - if (endpointURL.getProtocol().equalsIgnoreCase("https")) { - addEndpoint(Protocol.HTTP, endpointURL.getHost(), endpointURL.getPort(), true); - } else if (endpointURL.getProtocol().equalsIgnoreCase("http")) { - addEndpoint(Protocol.HTTP, endpointURL.getHost(), endpointURL.getPort(), false); - } else { + if (!endpointURL.getProtocol().equalsIgnoreCase("https") && + !endpointURL.getProtocol().equalsIgnoreCase("http")) { throw new IllegalArgumentException("Only HTTP and HTTPS protocols are supported"); } + + // Build endpoint URL preserving the path but ignoring query parameters + StringBuilder sb = new StringBuilder(); + sb.append(endpointURL.getProtocol().toLowerCase()); + sb.append("://"); + sb.append(endpointURL.getHost()); + if (endpointURL.getPort() > 0) { + sb.append(":").append(endpointURL.getPort()); + } + String path = endpointURL.getPath(); + if (path != null && !path.isEmpty()) { + sb.append(path); + } + this.endpoints.add(sb.toString()); } catch (MalformedURLException e) { throw new IllegalArgumentException("Endpoint should be a valid URL string, but was " + endpoint, e); } 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 49d12098e..c8572e3cc 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -1120,6 +1120,71 @@ public void testSNIWithCloud() throws Exception { } } + @Test(groups = {"integration"}) + public void testEndpointUrlPathIsPreserved() throws Exception { + if (isCloud()) { + return; // mocked server + } + + int serverPort = new Random().nextInt(1000) + 10000; + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().port(serverPort) + .notifier(new Slf4jNotifier(true))); + mockServer.start(); + + try { + // Setup stubs for two virtual ClickHouse instances behind a reverse proxy + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales/db")) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"100\", \"read_rows\": \"10\"}")).build()); + + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing/db")) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"200\", \"read_rows\": \"20\"}")).build()); + + // Test sales virtual instance + try (Client salesClient = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort + "/sales/db") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = salesClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 100); + } + } + + // Test billing virtual instance - also verify query parameters in URL are ignored + try (Client billingClient = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort + "/billing/db?ignored_param=value") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = billingClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 200); + } + + // Verify that ignored_param is not in the request URL + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db")) + .withoutQueryParam("ignored_param")); + } + + // Verify requests were made to the correct paths + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/sales/db"))); + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db"))); + + } finally { + mockServer.stop(); + } + } + protected Client.Builder newClient() { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); boolean isSecure = isCloud(); From 99dfb3fbde61afe13560ed2c7c02427b9e00e798 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 11 Dec 2025 19:56:41 -0800 Subject: [PATCH 02/11] jdbc preserves path of endpoint --- .../jdbc/internal/JdbcConfiguration.java | 54 ++++++++++-- .../com/clickhouse/jdbc/ConnectionTest.java | 86 +++++++++++++++++++ .../jdbc/internal/JdbcConfigurationTest.java | 6 +- 3 files changed, 134 insertions(+), 12 deletions(-) diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java index fe601e51f..bc981f4b1 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java @@ -188,17 +188,53 @@ private Map parseUrl(String url) throws SQLException { if (uri.getAuthority().contains(",")) { throw new SQLException("Multiple endpoints not supported"); } - properties.put(PARSE_URL_CONN_URL_PROP, uri.getScheme() + "://" - + uri.getRawAuthority()); // will be parsed again later - if (uri.getPath() != null - && !uri.getPath().trim().isEmpty() - && !"/".equals(uri.getPath())) - { - properties.put( - ClientConfigProperties.DATABASE.getKey(), - uri.getPath().substring(1)); + // Parse path: last segment is database name, everything before is HTTP path + // Example: /proxy/path/mydb -> httpPath=/proxy/path, database=mydb + // Example: /mydb -> httpPath=empty, database=mydb + // Example: /sales/db -> httpPath=/sales, database=db + // Use raw path for splitting to avoid issues with URL-encoded slashes (e.g., %2F) + String rawPath = uri.getRawPath(); + String httpPath = ""; + String database = null; + + if (rawPath != null && !rawPath.trim().isEmpty() && !"/".equals(rawPath)) { + // Remove leading slash for processing + String pathWithoutLeadingSlash = rawPath.startsWith("/") ? rawPath.substring(1) : rawPath; + int lastSlashIndex = pathWithoutLeadingSlash.lastIndexOf('/'); + + if (lastSlashIndex > 0) { + // Path has multiple segments: everything before last slash is HTTP path + httpPath = "/" + pathWithoutLeadingSlash.substring(0, lastSlashIndex); + // Decode the database name + try { + database = URLDecoder.decode(pathWithoutLeadingSlash.substring(lastSlashIndex + 1), StandardCharsets.UTF_8.name()); + } catch (UnsupportedEncodingException e) { + throw new SQLException("Failed to decode database name", e); + } + } else { + // Single segment: it's the database name, no HTTP path + // Decode the database name + try { + database = URLDecoder.decode(pathWithoutLeadingSlash, StandardCharsets.UTF_8.name()); + } catch (UnsupportedEncodingException e) { + throw new SQLException("Failed to decode database name", e); + } + } + } + + // Build connection URL with HTTP path preserved + StringBuilder connectionUrl = new StringBuilder(); + connectionUrl.append(uri.getScheme()).append("://").append(uri.getRawAuthority()); + if (!httpPath.isEmpty()) { + connectionUrl.append(httpPath); } + properties.put(PARSE_URL_CONN_URL_PROP, connectionUrl.toString()); + + if (database != null && !database.trim().isEmpty()) { + properties.put(ClientConfigProperties.DATABASE.getKey(), database); + } + if (uri.getQuery() != null && !uri.getQuery().trim().isEmpty()) { for (String pair : uri.getRawQuery().split("&")) { try { diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java index 9da719d0e..1c4810f29 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java @@ -967,4 +967,90 @@ public void testUseUserTimeZone() throws Exception { } } + + @Test(groups = {"integration"}) + public void testEndpointUrlPathIsPreserved() throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + // From wireshark dump as C Array - response for SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1 + char selectServerInfo[] = { + 0x03, 0x04, 0x75, 0x73, 0x65, 0x72, 0x08, 0x74, + 0x69, 0x6d, 0x65, 0x7a, 0x6f, 0x6e, 0x65, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x06, + 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x06, 0x53, + 0x74, 0x72, 0x69, 0x6e, 0x67, 0x06, 0x53, 0x74, + 0x72, 0x69, 0x6e, 0x67, 0x07, 0x64, 0x65, 0x66, + 0x61, 0x75, 0x6c, 0x74, 0x03, 0x55, 0x54, 0x43, + 0x0b, 0x32, 0x34, 0x2e, 0x33, 0x2e, 0x31, 0x2e, + 0x32, 0x36, 0x37, 0x32}; + + char select1Res[] = { + 0x01, 0x01, 0x31, 0x05, 0x55, 0x49, 0x6e, 0x74, + 0x38, 0x01}; + + // URL format: jdbc:clickhouse://host:port/http_path/database + // For /sales/db: http_path=/sales, database=db + // For /billing/db: http_path=/billing, database=db + + // Setup stubs for sales virtual instance (path: /sales) + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales")) + .withRequestBody(WireMock.matching(".*SELECT 1.*")) + .willReturn(WireMock.ok(new String(select1Res)) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"100\", \"read_rows\": \"10\"}")).build()); + + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales")) + .withRequestBody(WireMock.equalTo("SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1")) + .willReturn(WireMock.ok(new String(selectServerInfo)) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + // Setup stubs for billing virtual instance (path: /billing) + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing")) + .withRequestBody(WireMock.matching(".*SELECT 2.*")) + .willReturn(WireMock.ok(new String(select1Res)) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"200\", \"read_rows\": \"20\"}")).build()); + + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing")) + .withRequestBody(WireMock.equalTo("SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1")) + .willReturn(WireMock.ok(new String(selectServerInfo)) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + Properties properties = new Properties(); + properties.put("compress", "false"); + + // Test sales virtual instance: /sales/db means http_path=/sales, database=db + String salesJdbcUrl = "jdbc:clickhouse://localhost:" + mockServer.port() + "/sales/db"; + try (Connection conn = new ConnectionImpl(salesJdbcUrl, properties); + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("SELECT 1")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getInt(1), 1); + } + + // Test billing virtual instance: /billing/db means http_path=/billing, database=db + String billingJdbcUrl = "jdbc:clickhouse://localhost:" + mockServer.port() + "/billing/db"; + try (Connection conn = new ConnectionImpl(billingJdbcUrl, properties); + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("SELECT 2")) { + Assert.assertTrue(rs.next()); + } + + // Verify requests were made to the correct HTTP paths (/sales and /billing, not /sales/db) + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/sales"))); + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing"))); + + } finally { + mockServer.stop(); + } + } } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java index 26a09793d..ee1e94637 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java @@ -123,8 +123,8 @@ public void testParseURLValid(String jdbcURL, Properties properties, throws Exception { JdbcConfiguration configuration = new JdbcConfiguration(jdbcURL, properties); - assertEquals(configuration.getConnectionUrl(), connectionURL); - assertEquals(configuration.clientProperties, expectedClientProps); + assertEquals(configuration.getConnectionUrl(), connectionURL, "URL: " + jdbcURL); + assertEquals(configuration.clientProperties, expectedClientProps, "URL: " + jdbcURL); Client.Builder bob = new Client.Builder(); configuration.applyClientProperties(bob); Client client = bob.build(); @@ -144,7 +144,7 @@ public void testParseURLInvalid(String jdbcURL) { @Test(dataProvider = "validURLs") public void testAcceptsURLValid(String url) throws Exception { - Assert.assertTrue(JdbcConfiguration.acceptsURL(url)); + Assert.assertTrue(JdbcConfiguration.acceptsURL(url), "URL: " + url); } @Test From e37370703463a0dbb42ac39bf7d5ae15389ccd82 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 11 Dec 2025 20:06:16 -0800 Subject: [PATCH 03/11] Updated code comments --- .../clickhouse/jdbc/internal/JdbcConfiguration.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java index bc981f4b1..d3faac911 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java @@ -204,17 +204,19 @@ private Map parseUrl(String url) throws SQLException { int lastSlashIndex = pathWithoutLeadingSlash.lastIndexOf('/'); if (lastSlashIndex > 0) { - // Path has multiple segments: everything before last slash is HTTP path + // Path contains a slash (not at position 0), so it has at least two segments. + // Everything before the last slash becomes HTTP path, the last segment is the database. + // Example: "sales/db" -> httpPath="/sales", database="db" + // Example: "api/v1/clickhouse/mydb" -> httpPath="/api/v1/clickhouse", database="mydb" httpPath = "/" + pathWithoutLeadingSlash.substring(0, lastSlashIndex); - // Decode the database name try { database = URLDecoder.decode(pathWithoutLeadingSlash.substring(lastSlashIndex + 1), StandardCharsets.UTF_8.name()); } catch (UnsupportedEncodingException e) { throw new SQLException("Failed to decode database name", e); } } else { - // Single segment: it's the database name, no HTTP path - // Decode the database name + // No slash found (lastSlashIndex == -1), so it's a single segment representing the database name. + // Example: "mydb" -> httpPath="", database="mydb" try { database = URLDecoder.decode(pathWithoutLeadingSlash, StandardCharsets.UTF_8.name()); } catch (UnsupportedEncodingException e) { From bffb301d3d4bfaa1bdaafa0a6adfb5030c33859e Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 11 Dec 2025 20:10:34 -0800 Subject: [PATCH 04/11] some code cleanup --- .../clickhouse/jdbc/internal/JdbcConfiguration.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java index d3faac911..2eebe2560 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java @@ -209,19 +209,11 @@ private Map parseUrl(String url) throws SQLException { // Example: "sales/db" -> httpPath="/sales", database="db" // Example: "api/v1/clickhouse/mydb" -> httpPath="/api/v1/clickhouse", database="mydb" httpPath = "/" + pathWithoutLeadingSlash.substring(0, lastSlashIndex); - try { - database = URLDecoder.decode(pathWithoutLeadingSlash.substring(lastSlashIndex + 1), StandardCharsets.UTF_8.name()); - } catch (UnsupportedEncodingException e) { - throw new SQLException("Failed to decode database name", e); - } + database = URLDecoder.decode(pathWithoutLeadingSlash.substring(lastSlashIndex + 1), StandardCharsets.UTF_8); } else { // No slash found (lastSlashIndex == -1), so it's a single segment representing the database name. // Example: "mydb" -> httpPath="", database="mydb" - try { - database = URLDecoder.decode(pathWithoutLeadingSlash, StandardCharsets.UTF_8.name()); - } catch (UnsupportedEncodingException e) { - throw new SQLException("Failed to decode database name", e); - } + database = URLDecoder.decode(pathWithoutLeadingSlash, StandardCharsets.UTF_8); } } From 3d20d436ef3420f099fc82f6e4c5806016f8a16f Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 14 Jan 2026 15:48:21 -0800 Subject: [PATCH 05/11] Reworked addEndpoint a bit --- .../com/clickhouse/client/api/Client.java | 18 +- .../client/metadata/MetadataTests.java | 238 +++++++++--------- 2 files changed, 134 insertions(+), 122 deletions(-) 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 7efbab164..524e1fc89 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 @@ -330,15 +330,27 @@ public Builder addEndpoint(String endpoint) { * @param port - Endpoint port */ public Builder addEndpoint(Protocol protocol, String host, int port, boolean secure) { + return addEndpoint(protocol, host, port, secure, "/"); + } + + public Builder addEndpoint(Protocol protocol, String host, int port, boolean secure, String basePath) { ValidationUtils.checkNonBlank(host, "host"); ValidationUtils.checkNotNull(protocol, "protocol"); ValidationUtils.checkRange(port, 1, ValidationUtils.TCP_PORT_NUMBER_MAX, "port"); + StringBuilder endpointBuilder = new StringBuilder(); + endpointBuilder.append(protocol.toString().toLowerCase()); if (secure) { - // TODO: if secure init SSL context + endpointBuilder.append("s"); } - String endpoint = String.format("%s%s://%s:%d", protocol.toString().toLowerCase(), secure ? "s": "", host, port); - this.endpoints.add(endpoint); + endpointBuilder.append("://"); + endpointBuilder.append(host); + endpointBuilder.append(":"); + endpointBuilder.append(port); + endpointBuilder.append(basePath); + + this.endpoints.add(endpointBuilder.toString()); return this; + } 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 86722f8fd..275e158f5 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 @@ -1,119 +1,119 @@ -package com.clickhouse.client.metadata; - -import com.clickhouse.client.BaseIntegrationTest; -import com.clickhouse.client.ClickHouseClient; -import com.clickhouse.client.ClickHouseConfig; -import com.clickhouse.client.ClickHouseNode; -import com.clickhouse.client.ClickHouseNodeSelector; -import com.clickhouse.client.ClickHouseProtocol; -import com.clickhouse.client.ClickHouseRequest; -import com.clickhouse.client.ClickHouseResponse; -import com.clickhouse.client.ClickHouseServerForTest; -import com.clickhouse.client.api.Client; -import com.clickhouse.client.api.enums.Protocol; -import com.clickhouse.client.api.internal.ServerSettings; -import com.clickhouse.client.api.metadata.DefaultColumnToMethodMatchingStrategy; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.QuerySettings; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.ClickHouseDataType; -import com.clickhouse.data.ClickHouseRecord; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -import java.util.Iterator; -import java.util.List; - -public class MetadataTests extends BaseIntegrationTest { - - private Client client; - - @BeforeMethod(groups = { "integration" }) - public void setUp() { - client = newClient().build(); - } - - @Test(groups = { "integration" }) - public void testGetTableSchema() { - prepareDataSet("describe_table"); - - TableSchema schema = client.getTableSchema("describe_table", ClickHouseServerForTest.getDatabase()); - - Assert.assertEquals(schema.getTableName(), "describe_table"); - Assert.assertEquals(schema.getDatabaseName(), ClickHouseServerForTest.getDatabase()); - - Assert.assertEquals(schema.getColumns().size(), 2); - - List columns = schema.getColumns(); - Assert.assertEquals(columns.get(0).getColumnName(), "param1"); - Assert.assertEquals(columns.get(0).getDataType().name(), "UInt32"); - } - - @Test(groups = { "integration" }) - - public void testGetTableSchemaDifferentDb() throws Exception { - String table = "test_get_table_schema_different_db"; - String db = ClickHouseServerForTest.getDatabase() + "_schema_test" ; - try { - QuerySettings settings = new QuerySettings().setDatabase(db); - client.execute("DROP DATABASE IF EXISTS " + db).get().close(); - client.execute("CREATE DATABASE " + db).get().close(); - client.query("DROP TABLE IF EXISTS " + table, settings).get().close(); - client.query("CREATE TABLE " + table + " (rowId Int32) Engine=MergeTree ORDER BY ()", settings).get().close(); - TableSchema tableSchema = client.getTableSchema(table, db); - Assert.assertEquals(tableSchema.getColumnByName("rowId").getDataType(), ClickHouseDataType.Int32); - } finally { - client.execute("DROP DATABASE IF EXISTS " + db).get().close(); - } - } - - private void prepareDataSet(String tableName) { - - try { - String sql = "CREATE TABLE IF NOT EXISTS " + tableName + " (param1 UInt32, param2 UInt16) ENGINE = Memory"; - client.execute(sql).get(); - - sql = "INSERT INTO " + tableName + " VALUES (1, 2), (3, 4), (5, 6)"; - client.execute(sql).get(); - } catch (Exception e) { - Assert.fail("Failed to prepare data set", e); - } - } - - @Test(groups = {"integration"}, dataProvider = "testMatchingNormalizationData") - public void testDefaultColumnToMethodMatchingStrategy(String methodName, String columnName) { - methodName = DefaultColumnToMethodMatchingStrategy.INSTANCE.normalizeMethodName(methodName); - columnName = DefaultColumnToMethodMatchingStrategy.INSTANCE.normalizeColumnName(columnName); - Assert.assertEquals(methodName, columnName, "Method name: " + methodName + " Column name: " + columnName); - } - - @DataProvider(name = "testMatchingNormalizationData") - public Object[][] testMatchingNormalizationData() { - return new Object[][]{ - {"getLastName", "LastName"}, - {"getLastName", "last_name"}, - {"getLastName", "last.name"}, - {"setLastName", "last.name"}, - {"isLastUpdate", "last_update"}, - {"hasMore", "more"}, - {"getFIRST_NAME", "first_name"}, - {"setUPDATED_ON", "updated.ON"}, - {"getNUM_OF_TRIES", "num_of_tries"}, - {"gethas_more", "has_more"}, - - }; - } - - protected Client.Builder newClient() { - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - boolean isSecure = isCloud(); - return new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); - } -} +package com.clickhouse.client.metadata; + +import com.clickhouse.client.BaseIntegrationTest; +import com.clickhouse.client.ClickHouseClient; +import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseNodeSelector; +import com.clickhouse.client.ClickHouseProtocol; +import com.clickhouse.client.ClickHouseRequest; +import com.clickhouse.client.ClickHouseResponse; +import com.clickhouse.client.ClickHouseServerForTest; +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.metadata.DefaultColumnToMethodMatchingStrategy; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.ClickHouseRecord; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.Iterator; +import java.util.List; + +public class MetadataTests extends BaseIntegrationTest { + + private Client client; + + @BeforeMethod(groups = { "integration" }) + public void setUp() { + client = newClient().build(); + } + + @Test(groups = { "integration" }) + public void testGetTableSchema() { + prepareDataSet("describe_table"); + + TableSchema schema = client.getTableSchema("describe_table", ClickHouseServerForTest.getDatabase()); + + Assert.assertEquals(schema.getTableName(), "describe_table"); + Assert.assertEquals(schema.getDatabaseName(), ClickHouseServerForTest.getDatabase()); + + Assert.assertEquals(schema.getColumns().size(), 2); + + List columns = schema.getColumns(); + Assert.assertEquals(columns.get(0).getColumnName(), "param1"); + Assert.assertEquals(columns.get(0).getDataType().name(), "UInt32"); + } + + @Test(groups = { "integration" }) + + public void testGetTableSchemaDifferentDb() throws Exception { + String table = "test_get_table_schema_different_db"; + String db = ClickHouseServerForTest.getDatabase() + "_schema_test" ; + try { + QuerySettings settings = new QuerySettings().setDatabase(db); + client.execute("DROP DATABASE IF EXISTS " + db).get().close(); + client.execute("CREATE DATABASE " + db).get().close(); + client.query("DROP TABLE IF EXISTS " + table, settings).get().close(); + client.query("CREATE TABLE " + table + " (rowId Int32) Engine=MergeTree ORDER BY ()", settings).get().close(); + TableSchema tableSchema = client.getTableSchema(table, db); + Assert.assertEquals(tableSchema.getColumnByName("rowId").getDataType(), ClickHouseDataType.Int32); + } finally { + client.execute("DROP DATABASE IF EXISTS " + db).get().close(); + } + } + + private void prepareDataSet(String tableName) { + + try { + String sql = "CREATE TABLE IF NOT EXISTS " + tableName + " (param1 UInt32, param2 UInt16) ENGINE = Memory"; + client.execute(sql).get(); + + sql = "INSERT INTO " + tableName + " VALUES (1, 2), (3, 4), (5, 6)"; + client.execute(sql).get(); + } catch (Exception e) { + Assert.fail("Failed to prepare data set", e); + } + } + + @Test(groups = {"integration"}, dataProvider = "testMatchingNormalizationData") + public void testDefaultColumnToMethodMatchingStrategy(String methodName, String columnName) { + methodName = DefaultColumnToMethodMatchingStrategy.INSTANCE.normalizeMethodName(methodName); + columnName = DefaultColumnToMethodMatchingStrategy.INSTANCE.normalizeColumnName(columnName); + Assert.assertEquals(methodName, columnName, "Method name: " + methodName + " Column name: " + columnName); + } + + @DataProvider(name = "testMatchingNormalizationData") + public Object[][] testMatchingNormalizationData() { + return new Object[][]{ + {"getLastName", "LastName"}, + {"getLastName", "last_name"}, + {"getLastName", "last.name"}, + {"setLastName", "last.name"}, + {"isLastUpdate", "last_update"}, + {"hasMore", "more"}, + {"getFIRST_NAME", "first_name"}, + {"setUPDATED_ON", "updated.ON"}, + {"getNUM_OF_TRIES", "num_of_tries"}, + {"gethas_more", "has_more"}, + + }; + } + + protected Client.Builder newClient() { + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + boolean isSecure = isCloud(); + return new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); + } +} From acd514b0327452ea2b5745ea45ae3f09cd4355c0 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 14 Jan 2026 16:40:15 -0800 Subject: [PATCH 06/11] added more tests. moved to using Endpoint in builder --- .../com/clickhouse/client/api/Client.java | 78 ++++++++++--------- .../client/api/transport/Endpoint.java | 4 + .../client/api/transport/HttpEndpoint.java | 49 ++++++++++++ .../jdbc/internal/JdbcConfiguration.java | 10 +-- .../com/clickhouse/jdbc/ConnectionTest.java | 43 +++++++++- .../jdbc/internal/JdbcConfigurationTest.java | 11 ++- 6 files changed, 144 insertions(+), 51 deletions(-) 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 524e1fc89..d22fcf5fd 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 @@ -63,7 +63,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -142,12 +141,12 @@ public class Client implements AutoCloseable { private int retries; private LZ4Factory lz4Factory = null; - private Client(Set endpoints, Map configuration, + private Client(Collection endpoints, Map configuration, ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy) { this(endpoints, configuration, sharedOperationExecutor, columnToMethodMatchingStrategy, null); } - private Client(Set endpoints, Map configuration, + private Client(Collection endpoints, Map configuration, ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy, Object metricsRegistry) { // Simple initialization this.configuration = ClientConfigProperties.parseConfigMap(configuration); @@ -171,16 +170,16 @@ private Client(Set endpoints, Map configuration, // Transport ImmutableList.Builder tmpEndpoints = ImmutableList.builder(); boolean initSslContext = false; - for (String ep : endpoints) { - try { - HttpEndpoint endpoint = new HttpEndpoint(ep); - if (endpoint.isSecure()) { + for (Endpoint ep : endpoints) { + if (ep instanceof HttpEndpoint) { + HttpEndpoint httpEndpoint = (HttpEndpoint) ep; + if (httpEndpoint.isSecure()) { initSslContext = true; } - LOG.debug("Adding endpoint: {}", endpoint); - tmpEndpoints.add(endpoint); - } catch (Exception e) { - throw new ClientException("Failed to add endpoint " + ep, e); + LOG.debug("Adding endpoint: {}", httpEndpoint); + tmpEndpoints.add(httpEndpoint); + } else { + throw new ClientException("Unsupported endpoint type: " + ep.getClass().getName()); } } @@ -259,7 +258,7 @@ public void close() { public static class Builder { - private Set endpoints; + private List endpoints; // Read-only configuration private Map configuration; @@ -268,7 +267,7 @@ public static class Builder { private ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy; private Object metricRegistry = null; public Builder() { - this.endpoints = new HashSet<>(); + this.endpoints = new ArrayList<>(); this.configuration = new HashMap<>(); for (ClientConfigProperties p : ClientConfigProperties.values()) { @@ -297,28 +296,33 @@ public Builder addEndpoint(String endpoint) { try { URL endpointURL = new URL(endpoint); - if (!endpointURL.getProtocol().equalsIgnoreCase("https") && - !endpointURL.getProtocol().equalsIgnoreCase("http")) { + String protocolStr = endpointURL.getProtocol(); + if (!protocolStr.equalsIgnoreCase("https") && + !protocolStr.equalsIgnoreCase("http")) { throw new IllegalArgumentException("Only HTTP and HTTPS protocols are supported"); } - // Build endpoint URL preserving the path but ignoring query parameters - StringBuilder sb = new StringBuilder(); - sb.append(endpointURL.getProtocol().toLowerCase()); - sb.append("://"); - sb.append(endpointURL.getHost()); - if (endpointURL.getPort() > 0) { - sb.append(":").append(endpointURL.getPort()); + boolean secure = protocolStr.equalsIgnoreCase("https"); + String host = endpointURL.getHost(); + if (host == null || host.isEmpty()) { + throw new IllegalArgumentException("Host cannot be empty in endpoint: " + endpoint); } + + int port = endpointURL.getPort(); + if (port == -1) { + // Default ports if not specified + port = secure ? 443 : 80; + } + String path = endpointURL.getPath(); - if (path != null && !path.isEmpty()) { - sb.append(path); + if (path == null || path.isEmpty()) { + path = "/"; } - this.endpoints.add(sb.toString()); + + return addEndpoint(Protocol.HTTP, host, port, secure, path); } catch (MalformedURLException e) { throw new IllegalArgumentException("Endpoint should be a valid URL string, but was " + endpoint, e); } - return this; } /** @@ -337,18 +341,18 @@ public Builder addEndpoint(Protocol protocol, String host, int port, boolean sec ValidationUtils.checkNonBlank(host, "host"); ValidationUtils.checkNotNull(protocol, "protocol"); ValidationUtils.checkRange(port, 1, ValidationUtils.TCP_PORT_NUMBER_MAX, "port"); - StringBuilder endpointBuilder = new StringBuilder(); - endpointBuilder.append(protocol.toString().toLowerCase()); - if (secure) { - endpointBuilder.append("s"); - } - endpointBuilder.append("://"); - endpointBuilder.append(host); - endpointBuilder.append(":"); - endpointBuilder.append(port); - endpointBuilder.append(basePath); + ValidationUtils.checkNotNull(basePath, "basePath"); - this.endpoints.add(endpointBuilder.toString()); + if (protocol == Protocol.HTTP) { + try { + HttpEndpoint httpEndpoint = new HttpEndpoint(host, port, secure, basePath); + this.endpoints.add(httpEndpoint); + } catch (MalformedURLException e) { + throw new IllegalArgumentException("Failed to create HttpEndpoint", e); + } + } else { + throw new IllegalArgumentException("Unsupported protocol: " + protocol); + } return this; } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java b/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java index 7063e2951..1de9e8ab8 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java @@ -8,4 +8,8 @@ public interface Endpoint { String getBaseURL(); + String getHost(); + + int getPort(); + } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java index f391dca7d..50d2a7311 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java @@ -16,12 +16,47 @@ public class HttpEndpoint implements Endpoint { private final boolean secure; + private final String host; + + private final int port; + + private final String path; + public HttpEndpoint(String uri) throws MalformedURLException { this.uri = URI.create(uri); this.url = this.uri.toURL(); this.baseURL = url.toString(); this.info = baseURL; this.secure = this.uri.getScheme().equalsIgnoreCase("https"); + this.host = this.url.getHost(); + this.port = this.url.getPort() != -1 ? this.url.getPort() : (this.secure ? 443 : 80); + this.path = this.uri.getPath() != null && !this.uri.getPath().isEmpty() ? this.uri.getPath() : "/"; + } + + public HttpEndpoint(String host, int port, boolean secure, String basePath) throws MalformedURLException { + this.host = host; + this.port = port; + this.secure = secure; + if (basePath != null && !basePath.isEmpty()) { + // Ensure basePath starts with / + this.path = basePath.startsWith("/") ? basePath : "/" + basePath; + } else { + this.path = "/"; + } + + StringBuilder uriBuilder = new StringBuilder(); + uriBuilder.append(secure ? "https" : "http"); + uriBuilder.append("://"); + uriBuilder.append(host); + uriBuilder.append(":"); + uriBuilder.append(port); + uriBuilder.append(this.path); + + String uriString = uriBuilder.toString(); + this.uri = URI.create(uriString); + this.url = this.uri.toURL(); + this.baseURL = url.toString(); + this.info = baseURL; } @Override @@ -29,6 +64,20 @@ public String getBaseURL() { return baseURL; } + @Override + public String getHost() { + return host; + } + + @Override + public int getPort() { + return port; + } + + public String getPath() { + return path; + } + public URL getURL() { return url; } diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java index 2eebe2560..2581cd545 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java @@ -189,11 +189,7 @@ private Map parseUrl(String url) throws SQLException { throw new SQLException("Multiple endpoints not supported"); } - // Parse path: last segment is database name, everything before is HTTP path - // Example: /proxy/path/mydb -> httpPath=/proxy/path, database=mydb - // Example: /mydb -> httpPath=empty, database=mydb - // Example: /sales/db -> httpPath=/sales, database=db - // Use raw path for splitting to avoid issues with URL-encoded slashes (e.g., %2F) + String rawPath = uri.getRawPath(); String httpPath = ""; String database = null; @@ -204,10 +200,6 @@ private Map parseUrl(String url) throws SQLException { int lastSlashIndex = pathWithoutLeadingSlash.lastIndexOf('/'); if (lastSlashIndex > 0) { - // Path contains a slash (not at position 0), so it has at least two segments. - // Everything before the last slash becomes HTTP path, the last segment is the database. - // Example: "sales/db" -> httpPath="/sales", database="db" - // Example: "api/v1/clickhouse/mydb" -> httpPath="/api/v1/clickhouse", database="mydb" httpPath = "/" + pathWithoutLeadingSlash.substring(0, lastSlashIndex); database = URLDecoder.decode(pathWithoutLeadingSlash.substring(lastSlashIndex + 1), StandardCharsets.UTF_8); } else { diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java index 1c4810f29..d149357e9 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java @@ -776,6 +776,45 @@ public void testSelectingDatabase() throws Exception { } } + @Test(groups = { "integration" }) + public void testConnectionWithSingleSegmentUrl() throws Exception { + if (isCloud()) { + return; // no need to test in cloud + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + // Create database db1 + Connection connCreate = this.getJdbcConnection(); + connCreate.createStatement().executeUpdate("CREATE DATABASE `db1`"); + + try { + Properties properties = new Properties(); + properties.put(ClientConfigProperties.USER.getKey(), "default"); + properties.put(ClientConfigProperties.PASSWORD.getKey(), ClickHouseServerForTest.getPassword()); + + // Test URL with single segment "/db1" - should be parsed as database name + String jdbcUrl = "jdbc:clickhouse://" + server.getHost() + ":" + server.getPort() + "/db1"; + try (Connection conn = new ConnectionImpl(jdbcUrl, properties); + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("SELECT database()")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getString(1), "db1"); + } + + // Verify that queries work correctly + try (Connection conn = new ConnectionImpl(jdbcUrl, properties); + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("SELECT 1")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getInt(1), 1); + } + } finally { + // Clean up: drop database db1 + connCreate.createStatement().executeUpdate("DROP DATABASE `db1`"); + connCreate.close(); + } + } + @Test(groups = { "integration" }) public void testUnwrapping() throws Exception { Connection conn = getJdbcConnection(); @@ -980,7 +1019,7 @@ public void testEndpointUrlPathIsPreserved() throws Exception { try { // From wireshark dump as C Array - response for SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1 - char selectServerInfo[] = { + char[] selectServerInfo = { 0x03, 0x04, 0x75, 0x73, 0x65, 0x72, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x7a, 0x6f, 0x6e, 0x65, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x06, @@ -991,7 +1030,7 @@ public void testEndpointUrlPathIsPreserved() throws Exception { 0x0b, 0x32, 0x34, 0x2e, 0x33, 0x2e, 0x31, 0x2e, 0x32, 0x36, 0x37, 0x32}; - char select1Res[] = { + char[] select1Res = { 0x01, 0x01, 0x31, 0x05, 0x55, 0x49, 0x6e, 0x74, 0x38, 0x01}; diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java index ee1e94637..11d385b93 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java @@ -18,6 +18,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; public class JdbcConfigurationTest { @@ -129,9 +130,13 @@ public void testParseURLValid(String jdbcURL, Properties properties, configuration.applyClientProperties(bob); Client client = bob.build(); assertEquals(client.getEndpoints().size(), 1); - assertEquals( - client.getEndpoints().iterator().next(), - connectionURL); + + String actualUrl = client.getEndpoints().iterator().next(); + if (actualUrl != null && Math.abs(actualUrl.length() - connectionURL.length()) == 1) { + assertTrue(actualUrl.contains(connectionURL)); + } else { + assertEquals(actualUrl, connectionURL); + } } @Test(dataProvider = "invalidURLs") From 9fd10adbe255b2e533284f552d4d86b598897956 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 14 Jan 2026 22:30:21 -0800 Subject: [PATCH 07/11] Added more tests for endpoint --- .../com/clickhouse/client/api/Client.java | 8 +- .../client/api/transport/HttpEndpoint.java | 36 +-- .../api/transport/HttpEndpointTest.java | 212 ++++++++++++++++++ 3 files changed, 225 insertions(+), 31 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java 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 d22fcf5fd..4fdfc8ae0 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 @@ -344,12 +344,8 @@ public Builder addEndpoint(Protocol protocol, String host, int port, boolean sec ValidationUtils.checkNotNull(basePath, "basePath"); if (protocol == Protocol.HTTP) { - try { - HttpEndpoint httpEndpoint = new HttpEndpoint(host, port, secure, basePath); - this.endpoints.add(httpEndpoint); - } catch (MalformedURLException e) { - throw new IllegalArgumentException("Failed to create HttpEndpoint", e); - } + HttpEndpoint httpEndpoint = new HttpEndpoint(host, port, secure, basePath); + this.endpoints.add(httpEndpoint); } else { throw new IllegalArgumentException("Unsupported protocol: " + protocol); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java index 50d2a7311..189b81e91 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java @@ -1,6 +1,7 @@ package com.clickhouse.client.api.transport; -import java.net.MalformedURLException; +import com.clickhouse.client.api.ClientMisconfigurationException; + import java.net.URI; import java.net.URL; @@ -22,18 +23,7 @@ public class HttpEndpoint implements Endpoint { private final String path; - public HttpEndpoint(String uri) throws MalformedURLException { - this.uri = URI.create(uri); - this.url = this.uri.toURL(); - this.baseURL = url.toString(); - this.info = baseURL; - this.secure = this.uri.getScheme().equalsIgnoreCase("https"); - this.host = this.url.getHost(); - this.port = this.url.getPort() != -1 ? this.url.getPort() : (this.secure ? 443 : 80); - this.path = this.uri.getPath() != null && !this.uri.getPath().isEmpty() ? this.uri.getPath() : "/"; - } - - public HttpEndpoint(String host, int port, boolean secure, String basePath) throws MalformedURLException { + public HttpEndpoint(String host, int port, boolean secure, String basePath){ this.host = host; this.port = port; this.secure = secure; @@ -44,18 +34,14 @@ public HttpEndpoint(String host, int port, boolean secure, String basePath) thro this.path = "/"; } - StringBuilder uriBuilder = new StringBuilder(); - uriBuilder.append(secure ? "https" : "http"); - uriBuilder.append("://"); - uriBuilder.append(host); - uriBuilder.append(":"); - uriBuilder.append(port); - uriBuilder.append(this.path); - - String uriString = uriBuilder.toString(); - this.uri = URI.create(uriString); - this.url = this.uri.toURL(); - this.baseURL = url.toString(); + // Use URI constructor to properly handle encoding of path segments + try { + this.uri = new URI(secure ? "https" : "http", null, host, port, path, null, null); + this.url = this.uri.toURL(); + } catch (Exception e) { + throw new ClientMisconfigurationException("Failed to create endpoint URL", e); + } + this.baseURL = uri.toString(); this.info = baseURL; } diff --git a/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java b/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java new file mode 100644 index 000000000..274f7ef39 --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java @@ -0,0 +1,212 @@ +package com.clickhouse.client.api.transport; + +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.net.URI; +import java.net.URL; + +public class HttpEndpointTest { + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_Http() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); + + Assert.assertEquals(endpoint.getHost(), "localhost"); + Assert.assertEquals(endpoint.getPort(), 8123); + Assert.assertFalse(endpoint.isSecure()); + Assert.assertEquals(endpoint.getPath(), "/clickhouse"); + Assert.assertTrue(endpoint.getBaseURL().contains("http://localhost:8123")); + Assert.assertTrue(endpoint.getBaseURL().contains("/clickhouse")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_Https() { + HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/api"); + + Assert.assertEquals(endpoint.getHost(), "example.com"); + Assert.assertEquals(endpoint.getPort(), 8443); + Assert.assertTrue(endpoint.isSecure()); + Assert.assertEquals(endpoint.getPath(), "/api"); + Assert.assertTrue(endpoint.getBaseURL().contains("https://example.com:8443")); + Assert.assertTrue(endpoint.getBaseURL().contains("/api")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_DefaultPortHttp() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 80, false, null); + + Assert.assertEquals(endpoint.getHost(), "localhost"); + Assert.assertEquals(endpoint.getPort(), 80); + Assert.assertFalse(endpoint.isSecure()); + Assert.assertEquals(endpoint.getPath(), "/"); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_DefaultPortHttps() { + HttpEndpoint endpoint = new HttpEndpoint("example.com", 443, true, null); + + Assert.assertEquals(endpoint.getHost(), "example.com"); + Assert.assertEquals(endpoint.getPort(), 443); + Assert.assertTrue(endpoint.isSecure()); + Assert.assertEquals(endpoint.getPath(), "/"); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_NullPath() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, null); + + Assert.assertEquals(endpoint.getPath(), "/"); + Assert.assertTrue(endpoint.getBaseURL().contains("http://localhost:8123")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_EmptyPath() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, ""); + + Assert.assertEquals(endpoint.getPath(), "/"); + Assert.assertTrue(endpoint.getBaseURL().contains("http://localhost:8123")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_PathWithoutLeadingSlash() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "clickhouse"); + + Assert.assertEquals(endpoint.getPath(), "/clickhouse"); + Assert.assertTrue(endpoint.getBaseURL().contains("/clickhouse")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_PathWithLeadingSlash() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); + + Assert.assertEquals(endpoint.getPath(), "/clickhouse"); + Assert.assertTrue(endpoint.getBaseURL().contains("/clickhouse")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_ComplexPath() { + HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/sales/db"); + + Assert.assertEquals(endpoint.getPath(), "/sales/db"); + Assert.assertTrue(endpoint.getBaseURL().contains("/sales/db")); + } + + @Test(groups = {"unit"}) + public void testGetURL() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); + + URL url = endpoint.getURL(); + Assert.assertNotNull(url); + Assert.assertEquals(url.getHost(), "localhost"); + Assert.assertEquals(url.getPort(), 8123); + Assert.assertTrue(url.getPath().contains("clickhouse")); + } + + @Test(groups = {"unit"}) + public void testGetURI() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); + + URI uri = endpoint.getURI(); + Assert.assertNotNull(uri); + Assert.assertEquals(uri.getHost(), "localhost"); + Assert.assertEquals(uri.getPort(), 8123); + Assert.assertTrue(uri.getPath().contains("clickhouse")); + } + + @Test(groups = {"unit"}) + public void testToString() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); + + String str = endpoint.toString(); + Assert.assertNotNull(str); + Assert.assertTrue(str.contains("localhost")); + Assert.assertTrue(str.contains("8123")); + Assert.assertTrue(str.contains("clickhouse")); + } + + @Test(groups = {"unit"}) + public void testImplementsEndpoint() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, null); + + Assert.assertTrue(endpoint instanceof Endpoint); + Endpoint endpointInterface = endpoint; + Assert.assertEquals(endpointInterface.getHost(), "localhost"); + Assert.assertEquals(endpointInterface.getPort(), 8123); + Assert.assertTrue(endpointInterface.getBaseURL().contains("localhost")); + Assert.assertTrue(endpointInterface.getBaseURL().contains("8123")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_PathWithSpaces() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/my path with spaces"); + + Assert.assertEquals(endpoint.getHost(), "localhost"); + Assert.assertEquals(endpoint.getPort(), 8123); + Assert.assertFalse(endpoint.isSecure()); + // Path should be stored as-is (decoded) + Assert.assertEquals(endpoint.getPath(), "/my path with spaces"); + // baseURL should have encoded spaces (%20) + Assert.assertTrue(endpoint.getBaseURL().contains("%20")); + Assert.assertTrue(endpoint.getBaseURL().contains("my")); + Assert.assertTrue(endpoint.getBaseURL().contains("path")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_PathWithSpacesWithoutLeadingSlash() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "my path with spaces"); + + Assert.assertEquals(endpoint.getHost(), "localhost"); + Assert.assertEquals(endpoint.getPort(), 8123); + // Path should have leading slash added + Assert.assertEquals(endpoint.getPath(), "/my path with spaces"); + // baseURL should have encoded spaces + Assert.assertTrue(endpoint.getBaseURL().contains("%20")); + } + + @Test(groups = {"unit"}) + public void testConstructorWithHostPortSecurePath_PathWithMultipleSpaces() { + HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/api/v1/my resource name"); + + Assert.assertEquals(endpoint.getHost(), "example.com"); + Assert.assertEquals(endpoint.getPort(), 8443); + Assert.assertTrue(endpoint.isSecure()); + // Path should be stored as-is (decoded) + Assert.assertEquals(endpoint.getPath(), "/api/v1/my resource name"); + // baseURL should have encoded spaces + Assert.assertTrue(endpoint.getBaseURL().contains("%20")); + Assert.assertTrue(endpoint.getBaseURL().contains("api")); + Assert.assertTrue(endpoint.getBaseURL().contains("v1")); + } + + @Test(groups = {"unit"}) + public void testGetURI_PathWithSpaces() { + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/my path"); + + URI uri = endpoint.getURI(); + Assert.assertNotNull(uri); + // URI.getPath() decodes spaces + Assert.assertTrue(uri.getPath().contains("my")); + Assert.assertTrue(uri.getPath().contains("path")); + Assert.assertEquals(endpoint.getPath(), "/my path"); + } + + @Test(groups = {"unit"}) + public void testIPv6Address() { + HttpEndpoint endpoint = new HttpEndpoint("[::1]", 8123, false, null); + + Assert.assertEquals(endpoint.getHost(), "[::1]"); + Assert.assertEquals(endpoint.getPort(), 8123); + Assert.assertEquals(endpoint.getPath(), "/"); + } + + @Test(groups = {"unit"}) + public void testIPv6AddressWithPath() { + HttpEndpoint endpoint = new HttpEndpoint("[2001:db8::1]", 8443, true, "/clickhouse"); + + Assert.assertEquals(endpoint.getHost(), "[2001:db8::1]"); + Assert.assertEquals(endpoint.getPort(), 8443); + Assert.assertTrue(endpoint.isSecure()); + Assert.assertEquals(endpoint.getPath(), "/clickhouse"); + } +} From c9c51bc0923842fe331f23696dee4be334c45cc2 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 14 Jan 2026 23:03:15 -0800 Subject: [PATCH 08/11] cleanup code and tests --- .../com/clickhouse/client/api/Client.java | 2 +- .../api/internal/HttpAPIClientHelper.java | 8 +- .../client/api/transport/Endpoint.java | 16 +- .../client/api/transport/HttpEndpoint.java | 31 +- .../api/transport/HttpEndpointTest.java | 295 ++++++++---------- 5 files changed, 152 insertions(+), 200 deletions(-) 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 4fdfc8ae0..b2143b0ce 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 @@ -2064,7 +2064,7 @@ protected int getOperationTimeout() { */ @Deprecated public Set getEndpoints() { - return endpoints.stream().map(Endpoint::getBaseURL).collect(Collectors.toSet()); + return endpoints.stream().map(endpoint -> endpoint.getURI().toString()).collect(Collectors.toSet()); } public String getUser() { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index de3f4de78..363222c26 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -422,7 +422,7 @@ public ClassicHttpResponse executeRequest(Endpoint server, Map r } URI uri; try { - URIBuilder uriBuilder = new URIBuilder(server.getBaseURL()); + URIBuilder uriBuilder = new URIBuilder(server.getURI()); addQueryParams(uriBuilder, requestConfig); uri = uriBuilder.normalizeSyntax().build(); } catch (URISyntaxException e) { @@ -473,15 +473,15 @@ public ClassicHttpResponse executeRequest(Endpoint server, Map r } catch (UnknownHostException e) { closeQuietly(httpResponse); - LOG.warn("Host '{}' unknown", server.getBaseURL()); + LOG.warn("Host '{}' unknown", server); throw e; } catch (ConnectException | NoRouteToHostException e) { closeQuietly(httpResponse); - LOG.warn("Failed to connect to '{}': {}", server.getBaseURL(), e.getMessage()); + LOG.warn("Failed to connect to '{}': {}", server, e.getMessage()); throw e; } catch (Exception e) { closeQuietly(httpResponse); - LOG.debug("Failed to execute request to '{}': {}", server.getBaseURL(), e.getMessage(), e); + LOG.debug("Failed to execute request to '{}': {}", server, e.getMessage(), e); throw e; } } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java b/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java index 1de9e8ab8..b82d44ecf 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/transport/Endpoint.java @@ -1,15 +1,29 @@ package com.clickhouse.client.api.transport; +import java.net.URI; + /** * Interface defining the behavior of transport endpoint. * It is transport responsibility to provide suitable implementation. */ public interface Endpoint { - String getBaseURL(); + /** + * Returns URI without query parameters + * @return endpoint url + */ + URI getURI(); + /** + * Returns hostname of target server + * @return dns hostname + */ String getHost(); + /** + * Returns port of target server + * @return port number + */ int getPort(); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java index 189b81e91..5f9de4e6f 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java @@ -7,11 +7,7 @@ public class HttpEndpoint implements Endpoint { - private final URI uri; // contains complete connection URL + parameters - - private final URL url; // only communication part - - private final String baseURL; + private final URI uri; // only communication part private final String info; @@ -23,31 +19,30 @@ public class HttpEndpoint implements Endpoint { private final String path; - public HttpEndpoint(String host, int port, boolean secure, String basePath){ + public HttpEndpoint(String host, int port, boolean secure, String path){ this.host = host; this.port = port; this.secure = secure; - if (basePath != null && !basePath.isEmpty()) { + if (path != null && !path.isEmpty()) { // Ensure basePath starts with / - this.path = basePath.startsWith("/") ? basePath : "/" + basePath; + this.path = path.startsWith("/") ? path : "/" + path; } else { this.path = "/"; } // Use URI constructor to properly handle encoding of path segments + // Encode path segments separately to preserve slashes try { - this.uri = new URI(secure ? "https" : "http", null, host, port, path, null, null); - this.url = this.uri.toURL(); + this.uri = new URI(secure ? "https" : "http", null, host, port, this.path, null, null); } catch (Exception e) { throw new ClientMisconfigurationException("Failed to create endpoint URL", e); } - this.baseURL = uri.toString(); - this.info = baseURL; + this.info = uri.toString(); } @Override - public String getBaseURL() { - return baseURL; + public URI getURI() { + return uri; } @Override @@ -64,14 +59,6 @@ public String getPath() { return path; } - public URL getURL() { - return url; - } - - public URI getURI() { - return uri; - } - public boolean isSecure() { return secure; } diff --git a/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java b/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java index 274f7ef39..8870cae00 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/transport/HttpEndpointTest.java @@ -1,212 +1,163 @@ package com.clickhouse.client.api.transport; import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.net.URI; -import java.net.URL; - public class HttpEndpointTest { - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_Http() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); - - Assert.assertEquals(endpoint.getHost(), "localhost"); - Assert.assertEquals(endpoint.getPort(), 8123); - Assert.assertFalse(endpoint.isSecure()); - Assert.assertEquals(endpoint.getPath(), "/clickhouse"); - Assert.assertTrue(endpoint.getBaseURL().contains("http://localhost:8123")); - Assert.assertTrue(endpoint.getBaseURL().contains("/clickhouse")); + @DataProvider(name = "basicEndpointConfigs") + public Object[][] basicEndpointConfigs() { + return new Object[][]{ + // host, port, secure, basePath, expectedHost, expectedPort, expectedSecure, expectedPath + {"localhost", 8123, false, "/clickhouse", "localhost", 8123, false, "/clickhouse"}, + {"example.com", 8443, true, "/api", "example.com", 8443, true, "/api"}, + {"localhost", 80, false, null, "localhost", 80, false, "/"}, + {"example.com", 443, true, null, "example.com", 443, true, "/"}, + {"localhost", 8123, false, "", "localhost", 8123, false, "/"}, + {"localhost", 8123, false, "clickhouse", "localhost", 8123, false, "/clickhouse"}, + {"example.com", 8443, true, "/sales/db", "example.com", 8443, true, "/sales/db"}, + }; } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_Https() { - HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/api"); - - Assert.assertEquals(endpoint.getHost(), "example.com"); - Assert.assertEquals(endpoint.getPort(), 8443); - Assert.assertTrue(endpoint.isSecure()); - Assert.assertEquals(endpoint.getPath(), "/api"); - Assert.assertTrue(endpoint.getBaseURL().contains("https://example.com:8443")); - Assert.assertTrue(endpoint.getBaseURL().contains("/api")); + @DataProvider(name = "pathsWithSpaces") + public Object[][] pathsWithSpaces() { + return new Object[][]{ + // host, port, secure, basePath, expectedPath, shouldContainEncodedSpaces + {"localhost", 8123, false, "/my path with spaces", "/my path with spaces", true}, + {"localhost", 8123, false, "my path with spaces", "/my path with spaces", true}, + {"example.com", 8443, true, "/api/v1/my resource name", "/api/v1/my resource name", true}, + }; } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_DefaultPortHttp() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 80, false, null); - - Assert.assertEquals(endpoint.getHost(), "localhost"); - Assert.assertEquals(endpoint.getPort(), 80); - Assert.assertFalse(endpoint.isSecure()); - Assert.assertEquals(endpoint.getPath(), "/"); + @DataProvider(name = "ipv6Addresses") + public Object[][] ipv6Addresses() { + return new Object[][]{ + // host, port, secure, basePath, expectedHost, expectedPort, expectedSecure, expectedPath + {"[::1]", 8123, false, null, "[::1]", 8123, false, "/"}, + {"[2001:db8::1]", 8443, true, "/clickhouse", "[2001:db8::1]", 8443, true, "/clickhouse"}, + }; } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_DefaultPortHttps() { - HttpEndpoint endpoint = new HttpEndpoint("example.com", 443, true, null); - - Assert.assertEquals(endpoint.getHost(), "example.com"); - Assert.assertEquals(endpoint.getPort(), 443); - Assert.assertTrue(endpoint.isSecure()); - Assert.assertEquals(endpoint.getPath(), "/"); + @DataProvider(name = "baseUrlConfigs") + public Object[][] baseUrlConfigs() { + return new Object[][]{ + // host, port, secure, basePath, expectedBaseUrl + {"localhost", 8123, false, null, "http://localhost:8123/"}, + {"localhost", 8123, false, "/", "http://localhost:8123/"}, + {"localhost", 8123, false, "/clickhouse", "http://localhost:8123/clickhouse"}, + {"example.com", 8443, true, "/api", "https://example.com:8443/api"}, + {"example.com", 443, true, "/sales/db", "https://example.com:443/sales/db"}, + {"localhost", 80, false, "path", "http://localhost:80/path"}, + }; } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_NullPath() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, null); - - Assert.assertEquals(endpoint.getPath(), "/"); - Assert.assertTrue(endpoint.getBaseURL().contains("http://localhost:8123")); - } + @Test(dataProvider = "basicEndpointConfigs") + public void testBasicEndpointCreation(String host, int port, boolean secure, String basePath, + String expectedHost, int expectedPort, boolean expectedSecure, String expectedPath) { + HttpEndpoint endpoint = new HttpEndpoint(host, port, secure, basePath); - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_EmptyPath() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, ""); - - Assert.assertEquals(endpoint.getPath(), "/"); - Assert.assertTrue(endpoint.getBaseURL().contains("http://localhost:8123")); + Assert.assertEquals(endpoint.getHost(), expectedHost, "Host mismatch"); + Assert.assertEquals(endpoint.getPort(), expectedPort, "Port mismatch"); + Assert.assertEquals(endpoint.isSecure(), expectedSecure, "Secure flag mismatch"); + Assert.assertEquals(endpoint.getPath(), expectedPath, "Path mismatch"); + Assert.assertNotNull(endpoint.getURI(), "URI should not be null"); + Assert.assertNotNull(endpoint.toString(), "toString should not be null"); } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_PathWithoutLeadingSlash() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "clickhouse"); - - Assert.assertEquals(endpoint.getPath(), "/clickhouse"); - Assert.assertTrue(endpoint.getBaseURL().contains("/clickhouse")); - } + @Test(dataProvider = "pathsWithSpaces") + public void testPathsWithSpaces(String host, int port, boolean secure, String basePath, + String expectedPath, boolean shouldContainEncodedSpaces) { + HttpEndpoint endpoint = new HttpEndpoint(host, port, secure, basePath); - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_PathWithLeadingSlash() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); - - Assert.assertEquals(endpoint.getPath(), "/clickhouse"); - Assert.assertTrue(endpoint.getBaseURL().contains("/clickhouse")); - } + Assert.assertEquals(endpoint.getPath(), expectedPath, "Path mismatch"); - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_ComplexPath() { - HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/sales/db"); - - Assert.assertEquals(endpoint.getPath(), "/sales/db"); - Assert.assertTrue(endpoint.getBaseURL().contains("/sales/db")); + // The URI/URL should properly encode spaces + String uriString = endpoint.getURI().toString(); + if (shouldContainEncodedSpaces) { + Assert.assertTrue(uriString.contains("%20"), "URI should contain encoded spaces: " + uriString); + } } - @Test(groups = {"unit"}) - public void testGetURL() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); - - URL url = endpoint.getURL(); - Assert.assertNotNull(url); - Assert.assertEquals(url.getHost(), "localhost"); - Assert.assertEquals(url.getPort(), 8123); - Assert.assertTrue(url.getPath().contains("clickhouse")); - } + @Test(dataProvider = "ipv6Addresses") + public void testIpv6Addresses(String host, int port, boolean secure, String basePath, + String expectedHost, int expectedPort, boolean expectedSecure, String expectedPath) { + HttpEndpoint endpoint = new HttpEndpoint(host, port, secure, basePath); - @Test(groups = {"unit"}) - public void testGetURI() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); - - URI uri = endpoint.getURI(); - Assert.assertNotNull(uri); - Assert.assertEquals(uri.getHost(), "localhost"); - Assert.assertEquals(uri.getPort(), 8123); - Assert.assertTrue(uri.getPath().contains("clickhouse")); + Assert.assertEquals(endpoint.getHost(), expectedHost, "Host mismatch"); + Assert.assertEquals(endpoint.getPort(), expectedPort, "Port mismatch"); + Assert.assertEquals(endpoint.isSecure(), expectedSecure, "Secure flag mismatch"); + Assert.assertEquals(endpoint.getPath(), expectedPath, "Path mismatch"); + Assert.assertNotNull(endpoint.getURI(), "URI should not be null"); } - @Test(groups = {"unit"}) - public void testToString() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/clickhouse"); - - String str = endpoint.toString(); - Assert.assertNotNull(str); - Assert.assertTrue(str.contains("localhost")); - Assert.assertTrue(str.contains("8123")); - Assert.assertTrue(str.contains("clickhouse")); - } + @Test(dataProvider = "baseUrlConfigs") + public void testBaseUrl(String host, int port, boolean secure, String basePath, String expectedBaseUrl) { + HttpEndpoint endpoint = new HttpEndpoint(host, port, secure, basePath); - @Test(groups = {"unit"}) - public void testImplementsEndpoint() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, null); - - Assert.assertTrue(endpoint instanceof Endpoint); - Endpoint endpointInterface = endpoint; - Assert.assertEquals(endpointInterface.getHost(), "localhost"); - Assert.assertEquals(endpointInterface.getPort(), 8123); - Assert.assertTrue(endpointInterface.getBaseURL().contains("localhost")); - Assert.assertTrue(endpointInterface.getBaseURL().contains("8123")); + Assert.assertEquals(endpoint.toString(), expectedBaseUrl, "toString should match baseURL"); } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_PathWithSpaces() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/my path with spaces"); - - Assert.assertEquals(endpoint.getHost(), "localhost"); - Assert.assertEquals(endpoint.getPort(), 8123); - Assert.assertFalse(endpoint.isSecure()); - // Path should be stored as-is (decoded) - Assert.assertEquals(endpoint.getPath(), "/my path with spaces"); - // baseURL should have encoded spaces (%20) - Assert.assertTrue(endpoint.getBaseURL().contains("%20")); - Assert.assertTrue(endpoint.getBaseURL().contains("my")); - Assert.assertTrue(endpoint.getBaseURL().contains("path")); + @Test + public void testSecureVsInsecureScheme() { + HttpEndpoint insecureEndpoint = new HttpEndpoint("localhost", 8123, false, null); + HttpEndpoint secureEndpoint = new HttpEndpoint("localhost", 8443, true, null); + + Assert.assertEquals(insecureEndpoint.getURI().getScheme(), "http", "Insecure endpoint should use http://"); + Assert.assertFalse(insecureEndpoint.isSecure(), "Insecure endpoint should return false for isSecure()"); + + Assert.assertEquals(secureEndpoint.getURI().getScheme(), "https", "Insecure endpoint should not use https://"); + Assert.assertTrue(secureEndpoint.isSecure(), "Secure endpoint should return true for isSecure()"); } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_PathWithSpacesWithoutLeadingSlash() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "my path with spaces"); - - Assert.assertEquals(endpoint.getHost(), "localhost"); - Assert.assertEquals(endpoint.getPort(), 8123); - // Path should have leading slash added - Assert.assertEquals(endpoint.getPath(), "/my path with spaces"); - // baseURL should have encoded spaces - Assert.assertTrue(endpoint.getBaseURL().contains("%20")); + @Test + public void testSpecialCharactersInPath() { + // Test various special characters that need encoding + HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/path/with/special?chars#and&more"); + + Assert.assertNotNull(endpoint.getURI(), "URI should be created despite special characters"); + + // The path should be stored as provided (normalized with leading slash) + Assert.assertEquals(endpoint.getPath(), "/path/with/special?chars#and&more"); } - @Test(groups = {"unit"}) - public void testConstructorWithHostPortSecurePath_PathWithMultipleSpaces() { - HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/api/v1/my resource name"); - - Assert.assertEquals(endpoint.getHost(), "example.com"); - Assert.assertEquals(endpoint.getPort(), 8443); - Assert.assertTrue(endpoint.isSecure()); - // Path should be stored as-is (decoded) - Assert.assertEquals(endpoint.getPath(), "/api/v1/my resource name"); - // baseURL should have encoded spaces - Assert.assertTrue(endpoint.getBaseURL().contains("%20")); - Assert.assertTrue(endpoint.getBaseURL().contains("api")); - Assert.assertTrue(endpoint.getBaseURL().contains("v1")); + @Test + public void testEmptyAndNullBasePath() { + HttpEndpoint nullPath = new HttpEndpoint("localhost", 8123, false, null); + HttpEndpoint emptyPath = new HttpEndpoint("localhost", 8123, false, ""); + + Assert.assertEquals(nullPath.getPath(), "/", "Null basePath should result in /"); + Assert.assertEquals(emptyPath.getPath(), "/", "Empty basePath should result in /"); + + Assert.assertEquals(nullPath.getURI().toString(), "http://localhost:8123/"); + Assert.assertEquals(emptyPath.getURI().toString(), "http://localhost:8123/"); } - @Test(groups = {"unit"}) - public void testGetURI_PathWithSpaces() { - HttpEndpoint endpoint = new HttpEndpoint("localhost", 8123, false, "/my path"); - - URI uri = endpoint.getURI(); - Assert.assertNotNull(uri); - // URI.getPath() decodes spaces - Assert.assertTrue(uri.getPath().contains("my")); - Assert.assertTrue(uri.getPath().contains("path")); - Assert.assertEquals(endpoint.getPath(), "/my path"); + @Test + public void testPathNormalization() { + // Path without leading slash should get one added + HttpEndpoint withoutSlash = new HttpEndpoint("localhost", 8123, false, "api/v1"); + HttpEndpoint withSlash = new HttpEndpoint("localhost", 8123, false, "/api/v1"); + + Assert.assertEquals(withoutSlash.getPath(), "/api/v1", "Path should be normalized with leading slash"); + Assert.assertEquals(withSlash.getPath(), "/api/v1", "Path with slash should remain unchanged"); } - @Test(groups = {"unit"}) - public void testIPv6Address() { - HttpEndpoint endpoint = new HttpEndpoint("[::1]", 8123, false, null); - - Assert.assertEquals(endpoint.getHost(), "[::1]"); - Assert.assertEquals(endpoint.getPort(), 8123); - Assert.assertEquals(endpoint.getPath(), "/"); + @Test + public void testMultiplePathSegments() { + HttpEndpoint endpoint = new HttpEndpoint("example.com", 8443, true, "/api/v1/resources/items"); + + Assert.assertEquals(endpoint.getPath(), "/api/v1/resources/items"); + Assert.assertEquals(endpoint.getURI().toString(), "https://example.com:8443/api/v1/resources/items"); } - @Test(groups = {"unit"}) - public void testIPv6AddressWithPath() { - HttpEndpoint endpoint = new HttpEndpoint("[2001:db8::1]", 8443, true, "/clickhouse"); - - Assert.assertEquals(endpoint.getHost(), "[2001:db8::1]"); - Assert.assertEquals(endpoint.getPort(), 8443); - Assert.assertTrue(endpoint.isSecure()); - Assert.assertEquals(endpoint.getPath(), "/clickhouse"); + @Test + public void testUtf8CharactersInPath() { + + String cyrillicPath = "/база/данных"; + HttpEndpoint cyrillicEndpoint = new HttpEndpoint("localhost", 8123, false, cyrillicPath); + Assert.assertEquals(cyrillicEndpoint.getPath(), cyrillicPath, "Cyrillic path should be preserved"); + Assert.assertTrue(cyrillicEndpoint.getURI().toASCIIString().contains("%"), + "Cyrillic path should be percent-encoded in ASCII representation"); } } From 62fd15e0dd6ad1dded6703cc8b51bac663535454 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 15 Jan 2026 04:18:26 -0800 Subject: [PATCH 09/11] reverted back endpoint list in builder to be set --- .../src/main/java/com/clickhouse/client/api/Client.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 b2143b0ce..350b566de 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 @@ -40,6 +40,7 @@ import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.ClickHouseFormat; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import net.jpountz.lz4.LZ4Factory; import org.apache.hc.core5.concurrent.DefaultThreadFactory; import org.apache.hc.core5.http.ClassicHttpResponse; @@ -63,6 +64,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -258,7 +260,7 @@ public void close() { public static class Builder { - private List endpoints; + private Set endpoints; // Read-only configuration private Map configuration; @@ -267,7 +269,7 @@ public static class Builder { private ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy; private Object metricRegistry = null; public Builder() { - this.endpoints = new ArrayList<>(); + this.endpoints = new HashSet<>(); this.configuration = new HashMap<>(); for (ClientConfigProperties p : ClientConfigProperties.values()) { From 2e3673bf484554b61f08f83302ce16a1be63920b Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 15 Jan 2026 04:47:22 -0800 Subject: [PATCH 10/11] removed default port value --- .../main/java/com/clickhouse/client/api/Client.java | 5 ++--- .../java/com/clickhouse/client/ClientTests.java | 13 +++++++++++++ 2 files changed, 15 insertions(+), 3 deletions(-) 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 350b566de..e76788388 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 @@ -311,9 +311,8 @@ public Builder addEndpoint(String endpoint) { } int port = endpointURL.getPort(); - if (port == -1) { - // Default ports if not specified - port = secure ? 443 : 80; + if (port <= 0) { + throw new ValidationUtils.SettingsValidationException("port", "Valid port must be specified"); } String path = endpointURL.getPath(); 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 310a4edcf..60f2bc59b 100644 --- a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java @@ -10,6 +10,7 @@ import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream; import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.internal.ValidationUtils; import com.clickhouse.client.api.metadata.DefaultColumnToMethodMatchingStrategy; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.QueryResponse; @@ -45,6 +46,7 @@ import static java.time.temporal.ChronoUnit.MILLIS; import static java.time.temporal.ChronoUnit.SECONDS; +import static org.testng.AssertJUnit.fail; public class ClientTests extends BaseIntegrationTest { private static final Logger LOGGER = LoggerFactory.getLogger(ClientTests.class); @@ -443,6 +445,17 @@ public void testServerSettings() throws Exception { } } + @Test(groups = {"integration"}) + public void testInvalidEndpoint() { + + try { + new Client.Builder().addEndpoint("http://localhost/default"); + fail("Exception expected"); + } catch (ValidationUtils.SettingsValidationException e) { + Assert.assertTrue(e.getMessage().contains("port")); + } + } + public boolean isVersionMatch(String versionExpression, Client client) { List serverVersion = client.queryAll("SELECT version()"); return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression); From 9d18d4b4da0ccf5cfe365c322091deba43fb051b Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 15 Jan 2026 04:52:05 -0800 Subject: [PATCH 11/11] added equals and hashcode to HTTPEndpoint --- .../clickhouse/client/api/transport/HttpEndpoint.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java index 5f9de4e6f..839b167d6 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/transport/HttpEndpoint.java @@ -67,4 +67,14 @@ public boolean isSecure() { public String toString() { return info; } + + @Override + public boolean equals(Object obj) { + return obj instanceof HttpEndpoint && uri.equals(((HttpEndpoint)obj).uri); + } + + @Override + public int hashCode() { + return uri.hashCode(); + } }