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() {
*
* - {@code http://localhost:8123}
* - {@code https://localhost:8443}
+ * - {@code http://localhost:8123/clickhouse} (with path for reverse proxy scenarios)
*
*
- * @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();
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..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
@@ -188,17 +188,47 @@ 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 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 {
+ // No slash found (lastSlashIndex == -1), so it's a single segment representing the database name.
+ // Example: "mydb" -> httpPath="", database="mydb"
+ database = URLDecoder.decode(pathWithoutLeadingSlash, StandardCharsets.UTF_8);
+ }
+ }
+
+ // 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