Skip to content

Repsect node-specific credentials #1282

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

Merged
merged 2 commits into from
Mar 16, 2023
Merged
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
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@
* error while converting Nested values to Java maps.
* incorrect algorithm extracted from PEM. [#1274](https://github.com/ClickHouse/clickhouse-java/issues/1274)
* transaction failure introduced 0.4.0.
* respect node-specific credentials. [#1114](https://github.com/ClickHouse/clickhouse-java/issues/1114)

## 0.4.1, 2023-02-19
### Breaking Changes
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -427,7 +427,12 @@ protected void checkSealed() {
* @return non-null client
*/
protected ClickHouseClient getClient() {
return client != null ? client : ClickHouseClient.newInstance(getServer().getProtocol());
if (client != null) {
return client;
}

ClickHouseNode node = getServer();
return ClickHouseClient.newInstance(node.getCredentials().orElse(null), node.getProtocol());
}

/**
Expand Down Expand Up @@ -540,7 +545,7 @@ public boolean hasOutputStream() {
public final ClickHouseNode getServer() {
ClickHouseNode node = serverRef.get();
if (node == null) {
node = server.apply(getConfig().getNodeSelector());
node = server.apply(getClient().getConfig().getNodeSelector());
if (!serverRef.compareAndSet(null, node)) {
node = serverRef.get();
}
Expand All @@ -556,13 +561,15 @@ public final ClickHouseNode getServer() {
public ClickHouseConfig getConfig() {
if (config == null) {
ClickHouseConfig clientConfig = getClient().getConfig();
if (options.isEmpty()) {
ClickHouseNode node = getServer();
if (options.isEmpty()
&& clientConfig.getDefaultCredentials().equals(node.getCredentials(clientConfig))) {
config = clientConfig;
} else {
Map<ClickHouseOption, Serializable> merged = new HashMap<>();
merged.putAll(clientConfig.getAllOptions());
merged.putAll(options);
config = new ClickHouseConfig(merged, clientConfig.getDefaultCredentials(),
config = new ClickHouseConfig(merged, node.getCredentials(clientConfig),
clientConfig.getNodeSelector(), clientConfig.getMetricRegistry().orElse(null));
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@
public class ClickHouseRequestTest {
@Test(groups = { "unit" })
public void testBuild() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
Assert.assertNotNull(request);

ClickHouseConfig config = request.getConfig();
Expand Down Expand Up @@ -85,6 +85,30 @@ public void testBuild() {
Assert.assertEquals(m.getStatements().get(0), sql);
}

@Test(groups = { "unit" })
public void testCredentials() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
Assert.assertNotNull(request.getConfig().getDefaultCredentials());
Assert.assertEquals(request.getConfig().getDefaultCredentials().getUserName(),
ClickHouseDefaults.USER.getDefaultValue());
Assert.assertEquals(request.getConfig().getDefaultCredentials().getPassword(),
ClickHouseDefaults.PASSWORD.getDefaultValue());

final String user = "somebody";
final String password = "seCrets";
request = ClickHouseClient.newInstance().read(ClickHouseNode.builder()
.credentials(ClickHouseCredentials.fromUserAndPassword(user, password)).build());
Assert.assertNotNull(request.getConfig().getDefaultCredentials());
Assert.assertEquals(request.getConfig().getDefaultCredentials().getUserName(), user);
Assert.assertEquals(request.getConfig().getDefaultCredentials().getPassword(), password);

request = ClickHouseClient.newInstance()
.read(ClickHouseNode.of("tcp://localhost/default?user=" + user + "&password=" + password));
Assert.assertNotNull(request.getConfig().getDefaultCredentials());
Assert.assertEquals(request.getConfig().getDefaultCredentials().getUserName(), user);
Assert.assertEquals(request.getConfig().getDefaultCredentials().getPassword(), password);
}

@Test(groups = { "unit" })
public void testConfigChangeListener() {
final ClickHouseConfig config = new ClickHouseConfig();
Expand All @@ -111,7 +135,7 @@ public void settingChanged(ClickHouseRequest<?> source, String setting, Serializ
}
};
final ClickHouseParameterizedQuery select3 = ClickHouseParameterizedQuery.of(config, "select 3");
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
request.setChangeListener(listener);
Assert.assertTrue(changedOptions.isEmpty(), "Should have no option changed");
Assert.assertTrue(changedProperties.isEmpty(), "Should have no property changed");
Expand Down Expand Up @@ -173,7 +197,7 @@ public void settingChanged(ClickHouseRequest<?> source, String setting, Serializ

@Test(groups = { "unit" })
public void testServerListener() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
final List<Object[]> serverChanges = new ArrayList<>();
request.setServerListener(
(currentServer, newServer) -> serverChanges
Expand All @@ -199,7 +223,7 @@ public void testServerListener() {

@Test(groups = { "unit" })
public void testCopy() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
request.compressServerResponse(true, ClickHouseCompression.BROTLI, 2);
request.decompressClientRequest(true, ClickHouseCompression.ZSTD, 5);
request.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0]))
Expand Down Expand Up @@ -251,7 +275,7 @@ public void testCopy() {

@Test(groups = { "unit" })
public void testFormat() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
Assert.assertEquals(request.getFormat(),
(ClickHouseFormat) ClickHouseDefaults.FORMAT.getEffectiveDefaultValue());
request.format(ClickHouseFormat.TabSeparatedRawWithNamesAndTypes);
Expand All @@ -274,7 +298,7 @@ public void testFormat() {
@Test(groups = { "unit" })
public void testGetSetting() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance()
.connect("http://localhost?custom_settings=a%3D1%2Cb%3D2");
.read("http://localhost?custom_settings=a%3D1%2Cb%3D2");
Assert.assertEquals(request.getSetting("a", boolean.class), true);
Assert.assertEquals(request.getSetting("a", Boolean.class), true);
Assert.assertEquals(request.getSetting("a", false), true);
Expand All @@ -290,7 +314,7 @@ public void testGetSetting() {

@Test(groups = { "unit" })
public void testInputData() throws IOException {
Mutation request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()).write();
Mutation request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build()).write();
Assert.assertEquals(request.getConfig().getFormat(), ClickHouseDataConfig.DEFAULT_FORMAT);
Assert.assertEquals(request.getConfig().getRequestCompressAlgorithm(), ClickHouseCompression.NONE);
Assert.assertEquals(request.getConfig().getRequestCompressLevel(),
Expand Down Expand Up @@ -356,7 +380,7 @@ public void testInputData() throws IOException {

@Test(groups = { "unit" })
public void testInputStreamAndCustomWriter() throws IOException {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
Assert.assertFalse(request.hasInputStream());
Assert.assertFalse(request.getInputStream().isPresent());
Assert.assertFalse(request.getWriter().isPresent());
Expand Down Expand Up @@ -401,7 +425,7 @@ public void testInputStreamAndCustomWriter() throws IOException {
public void testNamedParameters() {
// String sql = "select xxx from xxx settings max_execution_time =
// :max_execution_time";
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());

String sql = "select :a,:b,:a";
request.query(sql).params("1", "2");
Expand All @@ -410,7 +434,7 @@ public void testNamedParameters() {

@Test(groups = { "unit" })
public void testOptions() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());

Assert.assertEquals(request.options, Collections.emptyMap());
Properties props = new Properties();
Expand All @@ -430,7 +454,7 @@ public void testOptions() {
@Test(groups = { "unit" })
public void testParams() {
String sql = "select :one as one, :two as two, * from my_table where key=:key and arr[:idx] in numbers(:range)";
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build())
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build())
.query(sql);
Assert.assertEquals(request.getQuery(), sql);
request.params(ClickHouseByteValue.of(Byte.MIN_VALUE));
Expand Down Expand Up @@ -475,7 +499,7 @@ public void testParams() {

@Test(groups = { "unit" })
public void testSeal() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
request.compressServerResponse(true, ClickHouseCompression.BROTLI, 2);
request.decompressClientRequest(true, ClickHouseCompression.ZSTD, 5);
request.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0]))
Expand Down Expand Up @@ -504,7 +528,7 @@ public void testSeal() {
@Test(groups = { "unit" })
public void testSession() {
String sessionId = UUID.randomUUID().toString();
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
Assert.assertEquals(request.getSessionId().isPresent(), false);
Assert.assertEquals(request.getSessionId(), Optional.empty());
Assert.assertEquals(request.getConfig().isSessionCheck(), false);
Expand Down Expand Up @@ -563,7 +587,7 @@ public void testSession() {

@Test(groups = { "unit" })
public void testSettings() {
ClickHouseRequest<?> request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build());
ClickHouseRequest<?> request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build());
Assert.assertEquals(request.getStatements().size(), 0);
request.set("enable_optimize_predicate_expression", 1);
Assert.assertEquals(request.getStatements().size(), 1);
Expand All @@ -576,7 +600,7 @@ public void testSettings() {

@Test(groups = { "unit" })
public void testMutation() {
Mutation request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()).write();
Mutation request = ClickHouseClient.newInstance().read(ClickHouseNode.builder().build()).write();
request.table("test_table").format(ClickHouseFormat.Arrow).data(new ByteArrayInputStream(new byte[0]));

String expectedSql = "INSERT INTO test_table\n FORMAT Arrow";
Expand Down