Skip to content

Commit a36728e

Browse files
authored
Merge pull request #2592 from ClickHouse/jdbc_various_fixes_091225
[JDBC] Fixes
2 parents f000e1f + 50c6266 commit a36728e

File tree

8 files changed

+106
-20
lines changed

8 files changed

+106
-20
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -136,6 +136,7 @@ public class Client implements AutoCloseable {
136136
private final Map<ClickHouseDataType, Class<?>> typeHintMapping;
137137

138138
// Server context
139+
private String dbUser;
139140
private String serverVersion;
140141
private Object metricsRegistry;
141142
private int retries;
@@ -196,7 +197,7 @@ private Client(Set<String> endpoints, Map<String,String> configuration,
196197
}
197198

198199
this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown");
199-
200+
this.dbUser = configuration.getOrDefault(ClientConfigProperties.USER.getKey(), ClientConfigProperties.USER.getDefObjVal());
200201
this.typeHintMapping = (Map<ClickHouseDataType, Class<?>>) this.configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey());
201202
}
202203

@@ -208,7 +209,10 @@ public void loadServerInfo() {
208209
try (QueryResponse response = this.query("SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1").get()) {
209210
try (ClickHouseBinaryFormatReader reader = this.newBinaryFormatReader(response)) {
210211
if (reader.next() != null) {
211-
this.configuration.put(ClientConfigProperties.USER.getKey(), reader.getString("user"));
212+
String tmpDbUser = reader.getString("user");
213+
if (tmpDbUser != null && !tmpDbUser.isEmpty()) {
214+
this.dbUser = tmpDbUser;
215+
}
212216
this.configuration.put(ClientConfigProperties.SERVER_TIMEZONE.getKey(), reader.getString("timezone"));
213217
serverVersion = reader.getString("version");
214218
}
@@ -2041,7 +2045,7 @@ public Set<String> getEndpoints() {
20412045
}
20422046

20432047
public String getUser() {
2044-
return (String) this.configuration.get(ClientConfigProperties.USER.getKey());
2048+
return dbUser;
20452049
}
20462050

20472051
public String getServerVersion() {

client-v2/src/main/java/com/clickhouse/client/api/internal/CommonSettings.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -233,8 +233,10 @@ public void setNetworkTimeout(long timeout, ChronoUnit unit) {
233233
* @return timeout in ms.
234234
*/
235235
public Long getNetworkTimeout() {
236-
return (Long) getOption(ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getKey(),
237-
ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getDefaultValue());
236+
// Socket operation timeout must be integer because of OS interface
237+
// Network timeout may be something else in the future. So we need to cast it to Long.
238+
return ((Number) getOption(ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getKey(),
239+
ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getDefObjVal())).longValue();
238240
}
239241

240242

client-v2/src/test/java/com/clickhouse/client/ClientTests.java

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,14 +7,18 @@
77
import com.clickhouse.client.api.ConnectionReuseStrategy;
88
import com.clickhouse.client.api.command.CommandResponse;
99
import com.clickhouse.client.api.enums.Protocol;
10+
import com.clickhouse.client.api.insert.InsertSettings;
1011
import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
12+
import com.clickhouse.client.api.internal.ServerSettings;
1113
import com.clickhouse.client.api.metadata.DefaultColumnToMethodMatchingStrategy;
1214
import com.clickhouse.client.api.query.GenericRecord;
1315
import com.clickhouse.client.api.query.QueryResponse;
1416
import com.clickhouse.client.api.query.QuerySettings;
1517
import com.clickhouse.client.api.query.Records;
1618
import com.clickhouse.client.config.ClickHouseClientOption;
1719
import com.clickhouse.client.query.QueryTests;
20+
import com.clickhouse.data.ClickHouseColumn;
21+
import com.clickhouse.data.ClickHouseFormat;
1822
import com.clickhouse.data.ClickHouseVersion;
1923
import org.apache.commons.lang3.RandomStringUtils;
2024
import org.slf4j.Logger;
@@ -24,6 +28,7 @@
2428
import org.testng.annotations.Test;
2529
import org.testng.util.Strings;
2630

31+
import java.io.ByteArrayInputStream;
2732
import java.net.ConnectException;
2833
import java.util.Arrays;
2934
import java.util.HashMap;
@@ -412,6 +417,32 @@ public void testLogComment() throws Exception {
412417
}
413418
}
414419

420+
@Test(groups = {"integration"})
421+
public void testServerSettings() throws Exception {
422+
try (Client client = newClient().build()) {
423+
client.execute("DROP TABLE IF EXISTS server_settings_test_table");
424+
client.execute("CREATE TABLE server_settings_test_table (v Float) Engine MergeTree ORDER BY ()");
425+
426+
final String queryId = UUID.randomUUID().toString();
427+
InsertSettings insertSettings = new InsertSettings()
428+
.setQueryId(queryId)
429+
.serverSetting(ServerSettings.ASYNC_INSERT, "1")
430+
.serverSetting(ServerSettings.WAIT_ASYNC_INSERT, "1");
431+
432+
String csvData = "0.33\n0.44\n0.55\n";
433+
client.insert("server_settings_test_table", new ByteArrayInputStream(csvData.getBytes()), ClickHouseFormat.CSV, insertSettings).get().close();
434+
435+
client.execute("SYSTEM FLUSH LOGS").get().close();
436+
437+
List<GenericRecord> logRecords = client.queryAll("SELECT * FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + queryId + "' AND type = 'QueryFinish'");
438+
439+
GenericRecord record = logRecords.get(0);
440+
String settings = record.getString(record.getSchema().nameToColumnIndex("Settings"));
441+
Assert.assertTrue(settings.contains(ServerSettings.ASYNC_INSERT + "=1"));
442+
// Assert.assertTrue(settings.contains(ServerSettings.WAIT_ASYNC_INSERT + "=1")); // uncomment after server fix
443+
}
444+
}
445+
415446
public boolean isVersionMatch(String versionExpression, Client client) {
416447
List<GenericRecord> serverVersion = client.queryAll("SELECT version()");
417448
return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression);

client-v2/src/test/java/com/clickhouse/client/SettingsTests.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -105,6 +105,8 @@ void testQuerySettingsSpecific() throws Exception {
105105

106106
{
107107
final QuerySettings settings = new QuerySettings();
108+
Assert.assertEquals(settings.getNetworkTimeout().intValue(),
109+
(Integer) ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getDefObjVal());
108110
settings.setNetworkTimeout(10, ChronoUnit.SECONDS);
109111
Assert.assertEquals(settings.getNetworkTimeout(), TimeUnit.SECONDS.toMillis(10));
110112
}
@@ -164,6 +166,8 @@ public void testInsertSettingsSpecific() throws Exception {
164166

165167
{
166168
final InsertSettings settings = new InsertSettings();
169+
Assert.assertEquals(settings.getNetworkTimeout().intValue(),
170+
(Integer) ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getDefObjVal());
167171
settings.setNetworkTimeout(10, ChronoUnit.SECONDS);
168172
Assert.assertEquals(settings.getNetworkTimeout(), TimeUnit.SECONDS.toMillis(10));
169173
}

client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java

Lines changed: 5 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,7 @@
5252
import java.util.Collections;
5353
import java.util.List;
5454
import java.util.UUID;
55+
import java.util.concurrent.Executors;
5556
import java.util.concurrent.TimeUnit;
5657
import java.util.zip.GZIPOutputStream;
5758

@@ -104,7 +105,8 @@ protected Client.Builder newClient() {
104105
.useHttpCompression(useHttpCompression)
105106
.setDefaultDatabase(ClickHouseServerForTest.getDatabase())
106107
.serverSetting(ServerSettings.ASYNC_INSERT, "0")
107-
.serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1");
108+
.serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1")
109+
.setSharedOperationExecutor(Executors.newCachedThreadPool());
108110
}
109111

110112
@AfterMethod(groups = { "integration" })
@@ -234,7 +236,6 @@ public void testInsertingPOJOWithNullValueForNonNullableColumn() throws Exceptio
234236
try (InsertResponse response = client.insert(tableName, Collections.singletonList(pojo), settings).get(30, TimeUnit.SECONDS)) {
235237
fail("Should have thrown an exception");
236238
} catch (ClickHouseException e) {
237-
e.printStackTrace();
238239
assertTrue(e.getCause() instanceof IllegalArgumentException);
239240
}
240241
}
@@ -281,15 +282,12 @@ public void insertRawDataAsync(boolean async) throws Exception {
281282
writer.flush();
282283
client.insert(tableName, new ByteArrayInputStream(data.toByteArray()),
283284
ClickHouseFormat.TSV, localSettings).whenComplete((response, throwable) -> {
284-
OperationMetrics metrics = response.getMetrics();
285285
assertEquals((int)response.getWrittenRows(), 1000 );
286286

287287
List<GenericRecord> records = client.queryAll("SELECT * FROM " + tableName);
288288
assertEquals(records.size(), 1000);
289-
assertTrue(Thread.currentThread().getName()
290-
.startsWith(async ? "ForkJoinPool.commonPool" : "main"), "Threads starts with " + Thread.currentThread().getName());
291289
})
292-
.join(); // wait operation complete. only for tests
290+
.join().close(); // wait operation complete. only for tests
293291
}
294292

295293
@DataProvider
@@ -660,14 +658,10 @@ public void testCollectionInsert() throws Exception {
660658
out.write(row.getBytes());
661659
}
662660
}, ClickHouseFormat.JSONEachRow, new InsertSettings()).get()) {
663-
System.out.println("Rows written: " + response.getWrittenRows());
664661
}
665662

666663
List<GenericRecord> records = client.queryAll("SELECT * FROM \"" + tableName + "\"" );
667-
668-
for (GenericRecord record : records) {
669-
System.out.println("> " + record.getString(1) + ", " + record.getFloat(2) + ", " + record.getFloat(3));
670-
}
664+
assertEquals(records.size(), 4);
671665
}
672666

673667
// static {

jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -233,7 +233,8 @@ public static Object convert(Object value, Class<?> type, ClickHouseColumn colum
233233
} else if (type == String.class) {
234234
return value.toString();
235235
} else if (type == Boolean.class || type == boolean.class) {
236-
return Boolean.parseBoolean(value.toString());
236+
String str = value.toString();
237+
return !("false".equalsIgnoreCase(str) || "0".equalsIgnoreCase(str));
237238
} else if (type == Byte.class || type == byte.class) {
238239
return Byte.parseByte(value.toString());
239240
} else if (type == Short.class || type == short.class) {

jdbc-v2/src/main/java/com/clickhouse/jdbc/metadata/DatabaseMetaDataImpl.java

Lines changed: 19 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -895,7 +895,9 @@ private static String generateSqlTypeSizes(String columnName) {
895895
SQLType type = JdbcUtils.CLICKHOUSE_TYPE_NAME_TO_SQL_TYPE_MAP.get(typeName);
896896
if (type == null) {
897897
try {
898-
type = JdbcUtils.convertToSqlType(ClickHouseColumn.of("v1", typeName).getDataType());
898+
ClickHouseColumn c = ClickHouseColumn.of("v", typeName);
899+
ClickHouseDataType dt = c.getDataType();
900+
type = JdbcUtils.convertToSqlType(dt);
899901
} catch (Exception e) {
900902
log.error("Failed to convert column data type to SQL type: {}", typeName, e);
901903
type = JDBCType.OTHER; // In case of error, return SQL type 0
@@ -1092,8 +1094,23 @@ public ResultSet getTypeInfo() throws SQLException {
10921094
row.put("NULLABLE", nullability);
10931095
};
10941096

1097+
private static final Consumer<Map<String, Object>> TYPE_INFO_VALUE_FUNCTION = row -> {
1098+
String typeName = (String) row.get("TYPE_NAME");
1099+
SQLType type = JdbcUtils.CLICKHOUSE_TYPE_NAME_TO_SQL_TYPE_MAP.get(typeName);
1100+
if (type == null) {
1101+
try {
1102+
type = JdbcUtils.convertToSqlType(ClickHouseDataType.valueOf(typeName));
1103+
} catch (Exception e) {
1104+
log.error("Failed to convert column data type to SQL type: {}", typeName, e);
1105+
type = JDBCType.OTHER; // In case of error, return SQL type 0
1106+
}
1107+
}
1108+
1109+
row.put("DATA_TYPE", type.getVendorTypeNumber());
1110+
};
1111+
10951112
private static final List<Consumer<Map<String, Object>>> GET_TYPE_INFO_MUTATORS = Arrays.asList(
1096-
DATA_TYPE_VALUE_FUNCTION,
1113+
TYPE_INFO_VALUE_FUNCTION,
10971114
NULLABILITY_VALUE_FUNCTION
10981115
);
10991116

jdbc-v2/src/test/java/com/clickhouse/jdbc/metadata/DatabaseMetaDataTest.java

Lines changed: 34 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,11 @@
1919
import java.sql.Types;
2020
import java.util.Arrays;
2121
import java.util.Collections;
22+
import java.util.HashSet;
2223
import java.util.List;
2324
import java.util.Properties;
25+
import java.util.Set;
26+
import java.util.stream.Collectors;
2427

2528
import static org.testng.Assert.assertEquals;
2629
import static org.testng.Assert.assertFalse;
@@ -37,7 +40,7 @@ public void testGetColumns() throws Exception {
3740
final String tableName = "get_columns_metadata_test";
3841
try (Statement stmt = conn.createStatement()) {
3942
stmt.executeUpdate("" +
40-
"CREATE TABLE " + tableName + " (id Int32, name String, v1 Nullable(Int8)) " +
43+
"CREATE TABLE " + tableName + " (id Int32, name String NOT NULL, v1 Nullable(Int8), v2 Array(Int8)) " +
4144
"ENGINE MergeTree ORDER BY ()");
4245
}
4346

@@ -109,6 +112,7 @@ public void testGetColumns() throws Exception {
109112
assertEquals(rs.getInt("DATA_TYPE"), Types.INTEGER);
110113
assertEquals(rs.getObject("DATA_TYPE"), Types.INTEGER);
111114
assertEquals(rs.getString("TYPE_NAME"), "Int32");
115+
assertFalse(rs.getBoolean("NULLABLE"));
112116

113117
assertTrue(rs.next());
114118
assertEquals(rs.getString("TABLE_SCHEM"), getDatabase());
@@ -117,6 +121,7 @@ public void testGetColumns() throws Exception {
117121
assertEquals(rs.getInt("DATA_TYPE"), Types.VARCHAR);
118122
assertEquals(rs.getObject("DATA_TYPE"), Types.VARCHAR);
119123
assertEquals(rs.getString("TYPE_NAME"), "String");
124+
assertFalse(rs.getBoolean("NULLABLE"));
120125

121126
assertTrue(rs.next());
122127
assertEquals(rs.getString("TABLE_SCHEM"), getDatabase());
@@ -125,6 +130,16 @@ public void testGetColumns() throws Exception {
125130
assertEquals(rs.getInt("DATA_TYPE"), Types.TINYINT);
126131
assertEquals(rs.getObject("DATA_TYPE"), Types.TINYINT);
127132
assertEquals(rs.getString("TYPE_NAME"), "Nullable(Int8)");
133+
assertTrue(rs.getBoolean("NULLABLE"));
134+
135+
assertTrue(rs.next());
136+
assertEquals(rs.getString("TABLE_SCHEM"), getDatabase());
137+
assertEquals(rs.getString("TABLE_NAME"), tableName);
138+
assertEquals(rs.getString("COLUMN_NAME"), "v2");
139+
assertEquals(rs.getInt("DATA_TYPE"), Types.ARRAY);
140+
assertEquals(rs.getObject("DATA_TYPE"), Types.ARRAY);
141+
assertEquals(rs.getString("TYPE_NAME"), "Array(Int8)");
142+
assertFalse(rs.getBoolean("NULLABLE"));
128143
}
129144
}
130145
}
@@ -450,6 +465,24 @@ public void testGetTypeInfo() throws Exception {
450465
}
451466
}
452467

468+
@Test(groups = {"integration"})
469+
public void testFindNestedTypes() throws Exception {
470+
try (Connection conn = getJdbcConnection()) {
471+
DatabaseMetaData dbmd = conn.getMetaData();
472+
try (ResultSet rs = dbmd.getTypeInfo()) {
473+
Set<String> nestedTypes = Arrays.stream(ClickHouseDataType.values())
474+
.filter(dt -> dt.isNested()).map(dt -> dt.name()).collect(Collectors.toSet());
475+
476+
while (rs.next()) {
477+
String typeName = rs.getString("TYPE_NAME");
478+
nestedTypes.remove(typeName);
479+
}
480+
481+
assertTrue(nestedTypes.isEmpty(), "Nested types " + nestedTypes + " not found");
482+
}
483+
}
484+
}
485+
453486
@Test(groups = { "integration" })
454487
public void testGetFunctions() throws Exception {
455488
if (ClickHouseVersion.of(getServerVersion()).check("(,23.8]")) {

0 commit comments

Comments
 (0)