diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java index 3c3fdcaab..7077d3ffa 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java @@ -359,11 +359,11 @@ public static void afterSuite() { clickhouseContainer.stop(); } - if (isCloud) { - if (!runQuery("DROP DATABASE IF EXISTS " + database)) { - LOGGER.warn("Failed to drop database for testing."); - } - } +// if (isCloud) { +// if (!runQuery("DROP DATABASE IF EXISTS " + database)) { +// LOGGER.warn("Failed to drop database for testing."); +// } +// } } public static String getDatabase() { 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 59b52b0a0..0e56fc617 100644 --- a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java @@ -23,6 +23,9 @@ public class ClientTests extends BaseIntegrationTest { @Test(dataProvider = "clientProvider") public void testAddSecureEndpoint(Client client) { + if (isCloud()) { + return; // will fail in other tests + } try { Optional genericRecord = client .queryAll("SELECT hostname()").stream().findFirst(); @@ -69,18 +72,14 @@ private static Client[] secureClientProvider() throws Exception { @Test public void testRawSettings() { - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - Client client = new Client.Builder() - .addEndpoint(node.toUri().toString()) - .setUsername("default") - .setPassword("") + Client client = newClient() .setOption("custom_setting_1", "value_1") .build(); client.execute("SELECT 1"); QuerySettings querySettings = new QuerySettings(); - querySettings.setOption("session_timezone", "Europe/Zurich"); + querySettings.serverSetting("session_timezone", "Europe/Zurich"); try (Records response = client.queryRecords("SELECT timeZone(), serverTimeZone()", querySettings).get(10, TimeUnit.SECONDS)) { @@ -91,7 +90,6 @@ public void testRawSettings() { Assert.assertEquals("UTC", record.getString(2)); }); } catch (Exception e) { - e.printStackTrace(); Assert.fail(e.getMessage()); } finally { client.close(); @@ -100,13 +98,7 @@ public void testRawSettings() { @Test public void testPing() { - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder() - .addEndpoint(node.toUri().toString()) - .setUsername("default") - .setPassword("") - .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "false").equals("true")) - .build()) { + try (Client client = newClient().build()) { Assert.assertTrue(client.ping()); } } @@ -122,4 +114,15 @@ public void testPingFailure() { Assert.assertFalse(client.ping(TimeUnit.SECONDS.toMillis(20))); } } + + 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()) + .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true")); + } } 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 c7badb6d2..5aaa8d424 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -155,6 +155,9 @@ public void closed(Socket socket) { @Test(groups = {"integration"}) public void testConnectionRequestTimeout() { + if (isCloud()) { + return; // mocked server + } int serverPort = new Random().nextInt(1000) + 10000; ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); @@ -197,6 +200,10 @@ public void testConnectionRequestTimeout() { @Test public void testConnectionReuseStrategy() { + if (isCloud()) { + return; // mocked server + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); try (Client client = new Client.Builder() @@ -218,6 +225,10 @@ public void testConnectionReuseStrategy() { @Test(groups = { "integration" }) public void testSecureConnection() { + if (isCloud()) { + return; // will fail in other tests + } + ClickHouseNode secureServer = getSecureServer(ClickHouseProtocol.HTTP); try (Client client = new Client.Builder() @@ -240,6 +251,10 @@ public void testSecureConnection() { @Test(groups = { "integration" }, dataProvider = "NoResponseFailureProvider") public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, ThrowingFunction function, boolean shouldFail) { + if (isCloud()) { + return; // mocked server + } + WireMockServer faultyServer = new WireMockServer( WireMockConfiguration .options().port(9090).notifier(new ConsoleNotifier(false))); faultyServer.start(); @@ -318,6 +333,10 @@ public static Object[][] noResponseFailureProvider() { @Test(groups = { "integration" }, dataProvider = "testServerErrorHandlingDataProvider") public void testServerErrorHandling(ClickHouseFormat format, boolean serverCompression, boolean useHttpCompression) { + if (isCloud()) { + return; // mocked server + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); try (Client client = new Client.Builder() .addEndpoint(server.getBaseUri()) @@ -442,6 +461,10 @@ public void testErrorWithSuccessfulResponse() { @Test(groups = { "integration" }, dataProvider = "testServerErrorsUncompressedDataProvider") public void testServerErrorsUncompressed(int code, String message, String expectedMessage) { + if (isCloud()) { + return; // mocked server + } + WireMockServer mockServer = new WireMockServer( WireMockConfiguration .options().port(9090).notifier(new ConsoleNotifier(false))); mockServer.start(); @@ -497,6 +520,10 @@ public static Object[][] testServerErrorsUncompressedDataProvider() { @Test(groups = { "integration" }) public void testAdditionalHeaders() { + if (isCloud()) { + return; // mocked server + } + WireMockServer mockServer = new WireMockServer( WireMockConfiguration .options().port(9090).notifier(new ConsoleNotifier(false))); mockServer.start(); @@ -538,6 +565,10 @@ public void testAdditionalHeaders() { @Test(groups = { "integration" }) public void testServerSettings() { + if (isCloud()) { + return; // mocked server + } + WireMockServer mockServer = new WireMockServer( WireMockConfiguration .options().port(9090).notifier(new ConsoleNotifier(false))); mockServer.start(); @@ -733,6 +764,10 @@ public void testSSLAuthentication_invalidConfig() throws Exception { @Test(groups = { "integration" }) public void testErrorWithSendProgressHeaders() throws Exception { + if (isCloud()) { + return; // mocked server + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) .setUsername("default") @@ -760,6 +795,9 @@ public void testErrorWithSendProgressHeaders() throws Exception { @Test(groups = { "integration" }, dataProvider = "testUserAgentHasCompleteProductName_dataProvider", dataProviderClass = HttpTransportTests.class) public void testUserAgentHasCompleteProductName(String clientName, Pattern userAgentPattern) throws Exception { + if (isCloud()) { + return; // mocked server + } ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); try (Client client = new Client.Builder() @@ -799,6 +837,10 @@ public static Object[][] testUserAgentHasCompleteProductName_dataProvider() { @Test(groups = { "integration" }) public void testBearerTokenAuth() throws Exception { + if (isCloud()) { + return; // mocked server + } + WireMockServer mockServer = new WireMockServer( WireMockConfiguration .options().port(9090).notifier(new ConsoleNotifier(false))); mockServer.start(); diff --git a/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java b/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java index 323f787b2..0af6fdc9e 100644 --- a/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/ProxyTests.java @@ -43,12 +43,20 @@ public void setUp() throws IOException { @AfterMethod(groups = { "integration" }) public void teardown() { + if (isCloud()) { + return; // nothing to stop + } + proxy.get().stop(); client.get().close(); } @Test(groups = { "integration" }) public void testSimpleQuery() throws Exception { + if (isCloud()) { + return; // to specific setup for cloud, may be later + } + client.set(clientBuilder(initProxy(), false).build()); addProxyStub(); @@ -58,6 +66,9 @@ public void testSimpleQuery() throws Exception { @Test(groups = { "integration" }) public void testInsert() throws Exception { + if (isCloud()) { + return; // to specific setup for cloud, may be later + } String tableName = "simple_pojo_disable_proxy_table"; String createSQL = SamplePOJO.generateTableCreateSQL(tableName); client.set(clientBuilder(initProxy(), false).build()); @@ -81,6 +92,10 @@ public void testInsert() throws Exception { @Test(groups = { "integration" }) public void testPrivateProxyWithoutAuth() { + if (isCloud()) { + return; // to specific setup for cloud, may be later + } + client.set(clientBuilder(initProxy(), true).build()); addPrivateProxyStub(); @@ -98,6 +113,10 @@ public void testPrivateProxyWithoutAuth() { @Test(groups = { "integration" }) public void testPrivateProxyWithCredentials() { + if (isCloud()) { + return; // to specific setup for cloud, may be later + } + client.set(clientBuilder(initProxy(), true) .setProxyCredentials("user", "pass").build()); addPrivateProxyStub(); @@ -112,6 +131,10 @@ public void testPrivateProxyWithCredentials() { @Test(groups = { "integration" }) public void testProxyWithCookies() { + if (isCloud()) { + return; // to specific setup for cloud, may be later + } + client.set(clientBuilder(initProxy(), true).build()); final int targetPort = getServer(ClickHouseProtocol.HTTP).getPort(); @@ -137,6 +160,10 @@ public void testProxyWithCookies() { @Test(groups = { "integration" }) public void testProxyWithDisabledCookies() { + if (isCloud()) { + return; // to specific setup for cloud, may be later + } + client.set(clientBuilder(initProxy(), true).setHttpCookiesEnabled(false).build()); final int targetPort = getServer(ClickHouseProtocol.HTTP).getPort(); diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java index 4d02b6b00..b3b654e80 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java @@ -3,6 +3,7 @@ import com.clickhouse.client.BaseIntegrationTest; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseProtocol; +import com.clickhouse.client.ClickHouseServerForTest; import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientException; import com.clickhouse.client.api.command.CommandResponse; @@ -67,12 +68,7 @@ public InsertTests(boolean useClientCompression, boolean useHttpCompression) { public void setUp() throws IOException { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); int bufferSize = (7 * 65500); - client = new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) - .setUsername("default") - .setPassword("") - .compressClientRequest(useClientCompression) - .useHttpCompression(useHttpCompression) + client = newClient() .setSocketSndbuf(bufferSize) .setSocketRcvbuf(bufferSize) .setClientNetworkBufferSize(bufferSize) @@ -83,6 +79,19 @@ public void setUp() throws IOException { .setQueryId(String.valueOf(UUID.randomUUID())); } + 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()) + .compressClientRequest(useClientCompression) + .useHttpCompression(useHttpCompression) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true")); + } + @AfterMethod(groups = { "integration" }) public void tearDown() { client.close(); diff --git a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java index 970022f87..267fd2813 100644 --- a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java @@ -5,6 +5,8 @@ import com.clickhouse.client.ClickHouseException; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseProtocol; +import com.clickhouse.client.ClickHouseServerForTest; +import com.clickhouse.client.ClientIntegrationTest; import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientException; import com.clickhouse.client.api.DataTypeUtils; @@ -32,6 +34,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.MappingIterator; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.StringEscapeUtils; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -111,14 +114,8 @@ public QueryTests(boolean useServerCompression, boolean useHttpCompression, bool @BeforeMethod(groups = {"integration"}) public void setUp() { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - client = new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) - .setUsername("default") - .setPassword("") - .compressClientRequest(false) - .compressServerResponse(useServerCompression) - .useHttpCompression(useHttpCompression) - .build(); + boolean isSecure = isCloud(); + client = newClient().build(); delayForProfiler(0); System.out.println("Real port: " + node.getPort()); @@ -234,8 +231,8 @@ public void testReadRecordsWithStreamAPI() throws Exception { for (int i = 0; i < tables; i++) { final String tableName = "a_" + i; expectedTableNames.add(tableName); - client.execute("DROP TABLE IF EXISTS default." + tableName); - client.execute("CREATE TABLE " + tableName +" (x UInt32) ENGINE = Memory"); + client.execute("DROP TABLE IF EXISTS " + tableName); + client.execute("CREATE TABLE " + tableName +" (x UInt32) ENGINE = MergeTree ORDER BY ()"); } Records records = client.queryRecords("SHOW TABLES").get(3, TimeUnit.SECONDS); @@ -333,8 +330,8 @@ public void testQueryAllTableNames() { for (int i = 0; i < tables; i++) { final String tableName = "a_" + i; expectedTableNames.add(tableName); - client.execute("DROP TABLE IF EXISTS default." + tableName); - client.execute("CREATE TABLE " + tableName +" (x UInt32) ENGINE = Memory"); + client.execute("DROP TABLE IF EXISTS " + tableName); + client.execute("CREATE TABLE " + tableName +" (x UInt32) ENGINE = MergeTree ORDER BY ()"); } List records = client.queryAll("SHOW TABLES"); @@ -1261,11 +1258,11 @@ public void testDataTypes(List columns, List> valueGene try { // Drop table - client.execute("DROP TABLE IF EXISTS default." + table).get(10, TimeUnit.SECONDS); + client.execute("DROP TABLE IF EXISTS " + table).get(10, TimeUnit.SECONDS); // Create table StringBuilder createStmtBuilder = new StringBuilder(); - createStmtBuilder.append("CREATE TABLE IF NOT EXISTS default.").append(table).append(" ("); + createStmtBuilder.append("CREATE TABLE IF NOT EXISTS ").append(table).append(" ("); for (String column : columns) { createStmtBuilder.append(column).append(", "); } @@ -1276,7 +1273,7 @@ public void testDataTypes(List columns, List> valueGene // Insert data StringBuilder insertStmtBuilder = new StringBuilder(); - insertStmtBuilder.append("INSERT INTO default.").append(table).append(" VALUES "); + insertStmtBuilder.append("INSERT INTO ").append(table).append(" VALUES "); insertStmtBuilder.append("("); for (Supplier valueSupplier : valueGenerators) { insertStmtBuilder.append(valueSupplier.get()).append(", "); @@ -1341,7 +1338,7 @@ public void testQueryMetrics() throws Exception { } StringBuilder insertStmtBuilder = new StringBuilder(); - insertStmtBuilder.append("INSERT INTO default.").append(DATASET_TABLE).append(" VALUES "); + insertStmtBuilder.append("INSERT INTO ").append(DATASET_TABLE).append(" VALUES "); final int rowsToInsert = 5; for (int i = 0; i < rowsToInsert; i++) { insertStmtBuilder.append("("); @@ -1403,11 +1400,11 @@ private List> prepareDataSet(String table, List colu try { // Drop table - client.execute("DROP TABLE IF EXISTS default." + table).get(10, TimeUnit.SECONDS); + client.execute("DROP TABLE IF EXISTS " + table).get(10, TimeUnit.SECONDS); // Create table StringBuilder createStmtBuilder = new StringBuilder(); - createStmtBuilder.append("CREATE TABLE IF NOT EXISTS default.").append(table).append(" ("); + createStmtBuilder.append("CREATE TABLE IF NOT EXISTS ").append(table).append(" ("); for (String column : columns) { createStmtBuilder.append(column).append(", "); } @@ -1417,7 +1414,7 @@ private List> prepareDataSet(String table, List colu // Insert data StringBuilder insertStmtBuilder = new StringBuilder(); - insertStmtBuilder.append("INSERT INTO default.").append(table).append(" VALUES "); + insertStmtBuilder.append("INSERT INTO ").append(table).append(" VALUES "); for (int i = 0; i < rows; i++) { insertStmtBuilder.append("("); Map values = writeValuesRow(insertStmtBuilder, columns, valueGenerators); @@ -1494,8 +1491,8 @@ void writeArrayValues(StringBuilder sb, Iterator values) { public void testQueryParams() throws Exception { final String table = "query_params_test_table"; - client.execute("DROP TABLE IF EXISTS default." + table).get(); - client.execute("CREATE TABLE default." + table + " (col1 UInt32, col2 String) ENGINE = MergeTree ORDER BY tuple()").get(); + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute("CREATE TABLE " + table + " (col1 UInt32, col2 String) ENGINE = MergeTree ORDER BY tuple()").get(); ByteArrayOutputStream insertData = new ByteArrayOutputStream(); try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(insertData))) { @@ -1540,8 +1537,8 @@ public void testQueryParams() throws Exception { public void testGetTableSchema() throws Exception { final String table = "table_schema_test"; - client.execute("DROP TABLE IF EXISTS default." + table).get(10, TimeUnit.SECONDS); - client.execute("CREATE TABLE default." + table + + client.execute("DROP TABLE IF EXISTS " + table).get(10, TimeUnit.SECONDS); + client.execute("CREATE TABLE " + table + " (col1 UInt32, col2 String) ENGINE = MergeTree ORDER BY tuple()").get(10, TimeUnit.SECONDS); TableSchema schema = client.getTableSchema(table); @@ -1769,7 +1766,7 @@ public void testOperationCustomRoles(String[] roles) throws Exception { return; } - final String password = UUID.randomUUID().toString(); + String password = "^" + RandomStringUtils.random(12, true, true) + "$"; final String rolesList = "\"" + Strings.join("\",\"", roles) + "\""; try (CommandResponse resp = client.execute("DROP ROLE IF EXISTS " + rolesList).get()) { } @@ -1777,7 +1774,7 @@ public void testOperationCustomRoles(String[] roles) throws Exception { } try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH sha256_password BY '" + password + "'" ).get()) { + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED BY '" + password + "'" ).get()) { } try (CommandResponse resp = client.execute("GRANT " + rolesList + " TO some_user").get()) { } @@ -1807,7 +1804,7 @@ public void testClientCustomRoles(String[] roles) throws Exception { return; } - final String password = UUID.randomUUID().toString(); + String password = "^" + RandomStringUtils.random(12, true, true) + "$"; final String rolesList = "\"" + Strings.join("\",\"", roles) + "\""; try (CommandResponse resp = client.execute("DROP ROLE IF EXISTS " + rolesList).get()) { } @@ -1922,14 +1919,16 @@ public void testGetColumnsByIndex() throws Exception { protected Client.Builder newClient() { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + boolean isSecure = isCloud(); return new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) .setUsername("default") - .setPassword("") + .setPassword(ClickHouseServerForTest.getPassword()) .compressClientRequest(false) - .compressServerResponse(true) + .compressServerResponse(useServerCompression) .useHttpCompression(useHttpCompression) .allowBinaryReaderToReuseBuffers(usePreallocatedBuffers) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true")); } @@ -1939,7 +1938,7 @@ public void testReadingSimpleAggregateFunction() throws Exception { client.execute("DROP TABLE IF EXISTS " + tableName).get(); client.execute("CREATE TABLE `" + tableName + "` " + "(idx UInt8, lowest_value SimpleAggregateFunction(min, UInt8), count SimpleAggregateFunction(sum, Int64), mp SimpleAggregateFunction(maxMap, Map(UInt8, UInt8))) " + - "ENGINE Memory;").get(); + "ENGINE MergeTree ORDER BY ();").get(); try (InsertResponse response = client.insert(tableName, new ByteArrayInputStream("1\t2\t3\t{1:2}".getBytes(StandardCharsets.UTF_8)), ClickHouseFormat.TSV).get(30, TimeUnit.SECONDS)) { @@ -1964,7 +1963,7 @@ public void testReadingEnumsAsStrings() throws Exception { client.execute("DROP TABLE IF EXISTS " + tableName).get(); client.execute("CREATE TABLE `" + tableName + "` " + "(idx UInt8, enum1 Enum8('a' = 1, 'b' = 2, 'c' = 3), enum2 Enum16('atch' = 1, 'batch' = 2, 'catch' = 3)) " + - "ENGINE Memory").get(); + "ENGINE MergeTree ORDER BY ()").get(); try (InsertResponse response = client.insert(tableName, new ByteArrayInputStream("1\ta\t2".getBytes(StandardCharsets.UTF_8)), ClickHouseFormat.TSV).get(30, TimeUnit.SECONDS)) { Assert.assertEquals(response.getWrittenRows(), 1); diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java index 5484db68d..40d10f23c 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java @@ -44,6 +44,7 @@ public StatementImpl(ConnectionImpl connection) throws SQLException { this.metrics = null; this.batch = new ArrayList<>(); this.schema = connection.getSchema();// remember DB name + LOG.info("Statement schema " + schema); this.maxRows = 0; } 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 ae5ff4d73..52a7c5c32 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ConnectionTest.java @@ -359,6 +359,9 @@ public void testSecureConnection() throws Exception { @Test(groups = { "integration" }) public void testSelectingDatabase() throws Exception { + if (isCloud()) { + return; // no need to test in cloud + } ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); Properties properties = new Properties(); properties.put(ClientConfigProperties.USER.getKey(), "default"); diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index 49120c11d..45ac99e99 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -35,15 +35,7 @@ public static void setUp() throws SQLException { } private Connection getConnection() throws SQLException { - Properties props = new Properties(); - props.put(ClientConfigProperties.USER.getKey(), ClientConfigProperties.USER.getDefaultValue()); - props.put(ClientConfigProperties.PASSWORD.getKey(), ClientConfigProperties.PASSWORD.getDefaultValue()); - try { - return DriverManager.getConnection(getEndpointString(isCloud()), props); - } catch (SQLException e) { - Driver.load(); - return DriverManager.getConnection(getEndpointString(isCloud()), props); - } + return getJdbcConnection(); } private int insertData(String sql) throws SQLException { @@ -59,7 +51,7 @@ public void testIntegerTypes() throws SQLException { runQuery("CREATE TABLE test_integers (order Int8, " + "int8 Int8, int16 Int16, int32 Int32, int64 Int64, int128 Int128, int256 Int256, " + "uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, uint128 UInt128, uint256 UInt256" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert minimum values insertData("INSERT INTO test_integers VALUES ( 1, " @@ -165,7 +157,7 @@ public void testIntegerTypes() throws SQLException { public void testDecimalTypes() throws SQLException { runQuery("CREATE TABLE test_decimals (order Int8, " + "dec Decimal(9, 2), dec32 Decimal32(4), dec64 Decimal64(8), dec128 Decimal128(18), dec256 Decimal256(18)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert minimum values insertData("INSERT INTO test_decimals VALUES ( 1, -9999999.99, -99999.9999, -9999999999.99999999, -99999999999999999999.999999999999999999, " + @@ -235,7 +227,7 @@ public void testDateTypes() throws SQLException { + "date Date, date32 Date32, " + "dateTime DateTime, dateTime32 DateTime32, " + "dateTime643 DateTime64(3), dateTime646 DateTime64(6), dateTime649 DateTime64(9)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert minimum values insertData("INSERT INTO test_dates VALUES ( 1, '1970-01-01', '1970-01-01', " + @@ -317,7 +309,7 @@ public void testStringTypes() throws SQLException { + "str String, fixed FixedString(6), " + "enum Enum8('a' = 6, 'b' = 7, 'c' = 8), enum8 Enum8('a' = 1, 'b' = 2, 'c' = 3), enum16 Enum16('a' = 1, 'b' = 2, 'c' = 3), " + "uuid UUID, ipv4 IPv4, ipv6 IPv6" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -374,7 +366,7 @@ public void testStringTypes() throws SQLException { public void testFloatTypes() throws SQLException { runQuery("CREATE TABLE test_floats (order Int8, " + "float32 Float32, float64 Float64" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert minimum values insertData("INSERT INTO test_floats VALUES ( 1, -3.4028233E38, -1.7976931348623157E308 )"); @@ -424,7 +416,7 @@ public void testFloatTypes() throws SQLException { public void testBooleanTypes() throws SQLException { runQuery("CREATE TABLE test_booleans (order Int8, " + "bool Boolean" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -457,7 +449,7 @@ public void testBooleanTypes() throws SQLException { public void testArrayTypes() throws SQLException { runQuery("CREATE TABLE test_arrays (order Int8, " + "array Array(Int8), arraystr Array(String)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -510,7 +502,7 @@ public void testArrayTypes() throws SQLException { public void testMapTypes() throws SQLException { runQuery("CREATE TABLE test_maps (order Int8, " + "map Map(String, Int8), mapstr Map(String, String)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -571,7 +563,7 @@ public void testNullableTypesSimpleStatement() throws SQLException { + "enum Nullable(Enum8('a' = 6, 'b' = 7, 'c' = 8)), enum8 Nullable(Enum8('a' = 1, 'b' = 2, 'c' = 3)), enum16 Nullable(Enum16('a' = 1, 'b' = 2, 'c' = 3)), " + "uuid Nullable(UUID), ipv4 Nullable(IPv4), ipv6 Nullable(IPv6), " + "float32 Nullable(Float32), float64 Nullable(Float64), " - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert null values insertData("INSERT INTO test_nullable VALUES ( 1, " @@ -599,7 +591,7 @@ public void testNullableTypesSimpleStatement() throws SQLException { public void testLowCardinalityTypeSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_low_cardinality (order Int8, " + "lowcardinality LowCardinality(String)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -628,7 +620,7 @@ public void testLowCardinalityTypeSimpleStatement() throws SQLException { public void testSimpleAggregateFunction() throws SQLException { runQuery("CREATE TABLE test_aggregate (order Int8, " + "int8 Int8" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -656,7 +648,7 @@ public void testSimpleAggregateFunction() throws SQLException { public void testNestedTypeSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_nested (order Int8, " + "nested Nested (int8 Int8, int16 Int16, int32 Int32, int64 Int64, int128 Int128, int256 Int256)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -697,7 +689,7 @@ public void testNestedTypeSimpleStatement() throws SQLException { public void testTupleTypeSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_tuple (order Int8, " + "tuple Tuple(int8 Int8, int16 Int16, int32 Int32, int64 Int64, int128 Int128, int256 Int256)" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -739,7 +731,7 @@ public void testTupleTypeSimpleStatement() throws SQLException { public void testJSONTypeSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_json (order Int8, " + "json JSON" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -768,7 +760,7 @@ public void testJSONTypeSimpleStatement() throws SQLException { public void testGeometricTypesSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_geometric (order Int8, " + "point Point, ring Ring, linestring LineString, multilinestring MultiLineString, polygon Polygon, multipolygon MultiPolygon" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -808,7 +800,7 @@ public void testGeometricTypesSimpleStatement() throws SQLException { public void testDynamicTypesSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_dynamic (order Int8, " + "dynamic Dynamic" - + ") ENGINE = Memory"); + + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values long seed = System.currentTimeMillis(); diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java index 314e7be92..a9a64518a 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java @@ -3,8 +3,10 @@ import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.client.api.query.GenericRecord; +import org.apache.commons.lang3.RandomStringUtils; import org.testng.Assert; import org.testng.annotations.Test; +import org.testng.util.Strings; import java.net.Inet4Address; import java.net.Inet6Address; @@ -347,12 +349,13 @@ private void testSettingRole() throws SQLException { List roles = Arrays.asList("role1", "role2"); + String userPass = "^" + RandomStringUtils.random(12, true, true) + "$"; try (ConnectionImpl conn = (ConnectionImpl) getJdbcConnection()) { try (Statement stmt = conn.createStatement()) { stmt.execute("DROP ROLE IF EXISTS " + String.join(", ", roles)); stmt.execute("DROP USER IF EXISTS some_user"); stmt.execute("CREATE ROLE " + String.join(", ", roles)); - stmt.execute("CREATE USER some_user IDENTIFIED WITH no_password"); + stmt.execute("CREATE USER some_user IDENTIFIED BY '" + userPass + "'"); stmt.execute("GRANT " + String.join(", ", roles) + " TO some_user"); stmt.execute("SET DEFAULT ROLE NONE TO some_user"); } @@ -360,7 +363,7 @@ private void testSettingRole() throws SQLException { Properties info = new Properties(); info.setProperty("user", "some_user"); - info.setProperty("password", ""); + info.setProperty("password", userPass); try (ConnectionImpl conn = new ConnectionImpl(getEndpointString(), info)) { GenericRecord record = conn.client.queryAll("SELECT currentRoles()").get(0);