Skip to content

[jdbc-v2] Adds default port and restricts multiple endpoints #2489

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@

import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.http.ClickHouseHttpProto;
import com.clickhouse.data.ClickHouseDataType;
import com.clickhouse.jdbc.Driver;
import com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -129,7 +130,14 @@ private static String createConnectionURL(String url, boolean ssl) throws SQLExc
: url;
try {
URI tmp = URI.create(adjustedURL);
return tmp.toASCIIString();
String asciiString = tmp.toASCIIString();
if (tmp.getPort() < 0) {
String port = ssl || adjustedURL.startsWith("https")
? ":" + ClickHouseHttpProto.DEFAULT_HTTPS_PORT
: ":" + ClickHouseHttpProto.DEFAULT_HTTP_PORT;
asciiString += port;
}
return asciiString;
} catch (IllegalArgumentException iae) {
throw new SQLException("Failed to parse URL '" + url + "'", iae);
}
Expand Down Expand Up @@ -171,6 +179,9 @@ private Map<String, String> parseUrl(String url) throws SQLException {
throw new SQLException(
"Invalid authority part JDBC URL '" + url + "'");
}
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
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
package com.clickhouse.jdbc.internal;

import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;

import org.testng.Assert;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
Expand All @@ -23,15 +25,17 @@ public class JdbcConfigurationTest {
new JdbcConfigurationTestData("jdbc:clickhouse://localhost"),
new JdbcConfigurationTestData("jdbc:clickhouse:http://localhost"),
new JdbcConfigurationTestData("jdbc:clickhouse:https://localhost")
.withExpectedConnectionURL("https://localhost"),
.withExpectedConnectionURL("https://localhost:8443"),
new JdbcConfigurationTestData("jdbc:clickhouse:https://localhost:8123")
.withExpectedConnectionURL("https://localhost:8123"),
new JdbcConfigurationTestData("jdbc:clickhouse://localhost")
.withAdditionalConnectionParameters(
Map.of(JdbcConfiguration.USE_SSL_PROP, "true"))
.withExpectedConnectionURL("https://localhost")
.withExpectedConnectionURL("https://localhost:8443")
.withAdditionalExpectedClientProperties(
Map.of("ssl", "true")),
new JdbcConfigurationTestData("jdbc:clickhouse://[::1]")
.withExpectedConnectionURL("http://[::1]"),
.withExpectedConnectionURL("http://[::1]:8123"),
new JdbcConfigurationTestData("jdbc:clickhouse://[::1]:8123")
.withExpectedConnectionURL("http://[::1]:8123"),
new JdbcConfigurationTestData("jdbc:clickhouse://localhost:8443")
Expand Down Expand Up @@ -111,14 +115,22 @@ public class JdbcConfigurationTest {
))
};

@SuppressWarnings("deprecation")
@Test(dataProvider = "validURLTestData")
public void testParseURLValid(String jdbcURL, Properties properties,
String connectionUrl, Map<String, String> expectedClientProps)
String connectionURL, Map<String, String> expectedClientProps)
throws Exception
{
JdbcConfiguration configuration = new JdbcConfiguration(jdbcURL, properties);
assertEquals(configuration.getConnectionUrl(), connectionUrl);
assertEquals(configuration.getConnectionUrl(), connectionURL);
assertEquals(configuration.clientProperties, expectedClientProps);
Client.Builder bob = new Client.Builder();
configuration.applyClientProperties(bob);
Client client = bob.build();
assertEquals(client.getEndpoints().size(), 1);
assertEquals(
client.getEndpoints().iterator().next(),
connectionURL);
}

@Test(dataProvider = "invalidURLs")
Expand Down Expand Up @@ -198,6 +210,9 @@ public Object[][] createInvalidConnectionURLs() {
{ "jdbc:clickhouse://foo.bar?x&y=z" },
{ "jdbc:clickhouse://foo.bar?x==&y=z" },
{ "jdbc:clickhouse://localhost?☺=value1" },
// multiple endpoints are invalid
{ "jdbc:clickhouse://foo,bar" },
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, multiple endpoints is what we do not plan to support:

  • JDBC has datasources for multiple endpoints
  • Client v2 has addEndpoint() that will be used to add more endpoints when we support them.

{ "jdbc:clickhouse://foo,bar.com:8123" },
};
}

Expand All @@ -210,7 +225,7 @@ private static final class JdbcConfigurationTestData {
Map.of( "user", "default", "password", "");

private static final String DEFAULT_EXPECTED_CONNECTION_URL =
"http://localhost";
"http://localhost:8123";

private final String url;
private final Properties connectionParameters;
Expand Down
Loading