diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java index 3bcc1c4cf..9e06714d2 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java @@ -413,13 +413,15 @@ protected ClickHouseRequest(ClickHouseClient client, Function & Serializable) server; this.serverRef = ref == null ? new AtomicReference<>(null) : ref; this.txRef = new AtomicReference<>(null); - this.sealed = sealed; this.externalTables = new LinkedList<>(); - this.options = options != null ? new HashMap<>(options) : new HashMap<>(); + this.options = new HashMap<>(); this.settings = new LinkedHashMap<>(client.getConfig().getCustomSettings()); + options(options); this.namedParameters = new HashMap<>(); + + this.sealed = sealed; } protected T changeProperty(String property, T oldValue, T newValue) { @@ -673,6 +675,47 @@ public ClickHouseParameterizedQuery getPreparedQuery() { return preparedQuery; } + /** + * Gets typed setting value. + * + * @param type of the setting value + * @param setting non-null setting key + * @param valueType non-null value type + * @return non-null value + */ + public T getSetting(String setting, Class valueType) { + Serializable value = settings.get(ClickHouseChecker.nonBlank(setting, PARAM_SETTING)); + return ClickHouseOption.fromString(value == null ? "" : value.toString(), valueType); + } + + /** + * Gets typed setting value. + * + * @param type of the setting value + * @param setting non-null setting key + * @param defaultValue non-null default value + * @return non-null value + */ + public T getSetting(String setting, T defaultValue) { + Serializable value = settings.get(ClickHouseChecker.nonBlank(setting, PARAM_SETTING)); + ClickHouseChecker.nonNull(defaultValue, "defaultValue"); + if (value == null) { + return defaultValue; + } + + return (T) ClickHouseOption.fromString(value.toString(), defaultValue.getClass()); + } + + /** + * Checks if a setting has been defined or not. + * + * @param setting setting + * @return true if the setting has been defined; false otherwise + */ + public boolean hasSetting(String setting) { + return settings.containsKey(setting); + } + /** * Gets immutable settings. * @@ -1087,6 +1130,26 @@ public SelfT options(Properties options) { return (SelfT) this; } + /** + * Checks if a option has been defined or not. + * + * @param option option + * @return true if the option has been defined; false otherwise + */ + public boolean hasOption(ClickHouseOption option) { + return options.containsKey(option); + } + + /** + * Checks if a option has been defined or not. + * + * @param key key of the option + * @return true if the option has been defined; false otherwise + */ + public boolean hasOption(String key) { + return options.containsKey(ClickHouseClientOption.fromKey(key)); + } + /** * Sets output file, to which response will be redirected. * @@ -1503,7 +1566,8 @@ public SelfT query(String query, String queryId) { } /** - * Sets all server settings. + * Sets all server settings. When {@code settings} is null or empty, it's same + * as {@link #clearSettings()}. * * @param settings settings * @return the request itself @@ -1523,7 +1587,7 @@ public SelfT settings(Map settings) { } else { Map m = new HashMap<>(); m.putAll(this.settings); - if (options != null) { + if (settings != null) { for (Entry e : settings.entrySet()) { set(e.getKey(), e.getValue()); m.remove(e.getKey()); @@ -1537,6 +1601,29 @@ public SelfT settings(Map settings) { return (SelfT) this; } + /** + * Clears server settings. + * + * @return the request itself + */ + @SuppressWarnings("unchecked") + public SelfT clearSettings() { + checkSealed(); + + if (!this.settings.isEmpty()) { + if (changeListener == null) { + this.settings.clear(); + resetCache(); + } else { + for (Iterator it = settings.keySet().iterator(); it.hasNext();) { + removeSetting(it.next()); + } + } + } + + return (SelfT) this; + } + /** * Clears session configuration including session id, session check(whether to * validate the id), and session timeout. Transaction will be removed as well. @@ -1911,11 +1998,11 @@ public SelfT reset() { this.options.clear(); this.settings.clear(); } else { - for (ClickHouseOption o : this.options.keySet().toArray(new ClickHouseOption[0])) { - removeOption(o); + for (Iterator it = this.options.keySet().iterator(); it.hasNext();) { + removeOption(it.next()); } - for (String s : this.settings.keySet().toArray(new String[0])) { - removeSetting(s); + for (Iterator it = this.settings.keySet().iterator(); it.hasNext();) { + removeSetting(it.next()); } } this.input = changeProperty(PROP_DATA, this.input, null); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java index 228cd65d4..31e7e02d5 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java @@ -8,6 +8,8 @@ import java.util.Optional; import java.util.TimeZone; +import com.clickhouse.client.ClickHouseChecker; + /** * This defines a configuration option. To put it in a nutshell, an option is * composed of key, default value(which implies type of the value) and @@ -71,35 +73,39 @@ public static Map toKeyValuePairs(String str) { * @param type of the value * @param value value in string format * @param clazz non-null class of the value - * @return typed value + * @return non-null typed value */ @SuppressWarnings("unchecked") - static T fromString(String value, Class clazz) { - if (value == null || clazz == null) { - throw new IllegalArgumentException("Non-null value and class are required"); + public static T fromString(String value, Class clazz) { + if (clazz == null) { + throw new IllegalArgumentException("Non-null value type is required"); + } else if (value == null) { + value = ""; } T result; if (clazz == boolean.class || clazz == Boolean.class) { final Boolean boolValue; - if ("1".equals(value) || "0".equals(value)) { + if (value.isEmpty()) { + boolValue = Boolean.FALSE; + } else if (value.length() == 1) { boolValue = "1".equals(value); } else { boolValue = Boolean.valueOf(value); } - result = clazz.cast(boolValue); + result = (T) boolValue; } else if (byte.class == clazz || Byte.class == clazz) { - result = clazz.cast(value.isEmpty() ? Byte.valueOf((byte) 0) : Byte.valueOf(value)); + result = (T) (value.isEmpty() ? Byte.valueOf((byte) 0) : Byte.valueOf(value)); } else if (short.class == clazz || Short.class == clazz) { - result = clazz.cast(value.isEmpty() ? Short.valueOf((short) 0) : Short.valueOf(value)); + result = (T) (value.isEmpty() ? Short.valueOf((short) 0) : Short.valueOf(value)); } else if (int.class == clazz || Integer.class == clazz) { - result = clazz.cast(value.isEmpty() ? Integer.valueOf(0) : Integer.valueOf(value)); + result = (T) (value.isEmpty() ? Integer.valueOf(0) : Integer.valueOf(value)); } else if (long.class == clazz || Long.class == clazz) { - result = clazz.cast(value.isEmpty() ? Long.valueOf(0L) : Long.valueOf(value)); + result = (T) (value.isEmpty() ? Long.valueOf(0L) : Long.valueOf(value)); } else if (float.class == clazz || Float.class == clazz) { - result = clazz.cast(value.isEmpty() ? Float.valueOf(0F) : Float.valueOf(value)); + result = (T) (value.isEmpty() ? Float.valueOf(0F) : Float.valueOf(value)); } else if (double.class == clazz || Double.class == clazz) { - result = clazz.cast(value.isEmpty() ? Double.valueOf(0D) : Double.valueOf(value)); + result = (T) (value.isEmpty() ? Double.valueOf(0D) : Double.valueOf(value)); } else if (Enum.class.isAssignableFrom(clazz)) { Enum enumValue = null; try { @@ -127,6 +133,27 @@ static T fromString(String value, Class clazz) { return result; } + /** + * Converts given string to typed value. When {@code value} is null or blank, + * {@code defaultValue} will be returned. + * + * @param type of the value + * @param value value in string format + * @param defaultValue non-null default value + * @return non-null typed value + */ + @SuppressWarnings("unchecked") + public static T fromString(String value, T defaultValue) { + if (defaultValue == null) { + throw new IllegalArgumentException("Non-null default value is required"); + } + if (ClickHouseChecker.isNullOrBlank(value)) { + return defaultValue; + } + + return (T) fromString(value, defaultValue.getClass()); + } + /** * Gets default value of the option. * diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java index 9f7825800..d78dce04a 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java @@ -116,11 +116,14 @@ public void settingChanged(ClickHouseRequest source, String setting, Serializ Assert.assertEquals(changedOptions.toArray(new Object[0]), new Object[][] { new Object[] { request, ClickHouseClientOption.ASYNC, null, false }, - new Object[] { request, ClickHouseClientOption.FORMAT, null, ClickHouseFormat.Arrow }, - new Object[] { request, ClickHouseClientOption.FORMAT, ClickHouseFormat.Arrow, + new Object[] { request, ClickHouseClientOption.FORMAT, null, + ClickHouseFormat.Arrow }, + new Object[] { request, ClickHouseClientOption.FORMAT, + ClickHouseFormat.Arrow, ClickHouseFormat.Avro }, new Object[] { request, ClickHouseClientOption.ASYNC, false, null }, - new Object[] { request, ClickHouseClientOption.FORMAT, ClickHouseFormat.Avro, null } }); + new Object[] { request, ClickHouseClientOption.FORMAT, + ClickHouseFormat.Avro, null } }); Assert.assertEquals(changedProperties.toArray(new Object[0]), new Object[][] { { request, ClickHouseRequest.PROP_QUERY, null, "select 1" }, { request, ClickHouseRequest.PROP_QUERY, "select 1", "select 2" }, @@ -160,15 +163,18 @@ public void testServerListener() { ClickHouseRequest request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()); final List serverChanges = new ArrayList<>(); request.setServerListener( - (currentServer, newServer) -> serverChanges.add(new Object[] { currentServer, newServer })); + (currentServer, newServer) -> serverChanges + .add(new Object[] { currentServer, newServer })); ClickHouseNode s11 = ClickHouseNode.of("http://node1"); ClickHouseNode s12 = ClickHouseNode.of("grpc://node1/system"); ClickHouseNode s21 = ClickHouseNode.of("tcp://node2"); ClickHouseNode s22 = ClickHouseNode.of("https://node2"); request.changeServer(request.getServer(), s11); - Assert.assertEquals(serverChanges.toArray(new Object[0]), new Object[][] { { ClickHouseNode.DEFAULT, s11 } }); + Assert.assertEquals(serverChanges.toArray(new Object[0]), + new Object[][] { { ClickHouseNode.DEFAULT, s11 } }); request.changeServer(ClickHouseNode.DEFAULT, s12); - Assert.assertEquals(serverChanges.toArray(new Object[0]), new Object[][] { { ClickHouseNode.DEFAULT, s11 } }); + Assert.assertEquals(serverChanges.toArray(new Object[0]), + new Object[][] { { ClickHouseNode.DEFAULT, s11 } }); request.changeServer(s11, s21); Assert.assertEquals(serverChanges.toArray(new Object[0]), new Object[][] { { ClickHouseNode.DEFAULT, s11 }, { s11, s21 } }); @@ -183,7 +189,8 @@ public void testCopy() { ClickHouseRequest request = ClickHouseClient.newInstance().connect(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])).build()); + request.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0])) + .build()); request.format(ClickHouseFormat.Avro); request.table("table1", "query_id1"); request.query("select :a", UUID.randomUUID().toString()); @@ -244,12 +251,30 @@ public void testFormat() { Assert.assertEquals(request.getFormat(), (ClickHouseFormat) ClickHouseDefaults.FORMAT.getEffectiveDefaultValue()); Assert.assertEquals(request.getInputFormat(), - ((ClickHouseFormat) ClickHouseDefaults.FORMAT.getEffectiveDefaultValue()).defaultInputFormat()); + ((ClickHouseFormat) ClickHouseDefaults.FORMAT.getEffectiveDefaultValue()) + .defaultInputFormat()); request.format(ClickHouseFormat.Arrow); Assert.assertEquals(request.getFormat(), ClickHouseFormat.Arrow); Assert.assertEquals(request.getInputFormat(), ClickHouseFormat.Arrow); } + @Test(groups = { "unit" }) + public void testGetSetting() { + ClickHouseRequest request = ClickHouseClient.newInstance() + .connect("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); + Assert.assertEquals(request.getSetting("a", int.class), 1); + Assert.assertEquals(request.getSetting("a", Integer.class), 1); + Assert.assertEquals(request.getSetting("a", 9), 1); + Assert.assertEquals(request.getSetting("b", "3"), "2"); + // request.settings(null); + request.clearSettings(); + Assert.assertTrue(request.getSettings().isEmpty()); + Assert.assertEquals(request.getSetting("b", 9), 9); + } + @Test(groups = { "unit" }) public void testNamedParameters() { // String sql = "select xxx from xxx settings max_execution_time = @@ -293,7 +318,8 @@ public void testParams() { "select -128 as one, NULL as two, * from my_table where key=NULL and arr[NULL] in numbers(NULL)"); request.params(ClickHouseStringValue.of(""), - ClickHouseDateTimeValue.of("2012-12-12 12:23:34.56789", 2, ClickHouseValues.UTC_TIMEZONE), + ClickHouseDateTimeValue.of("2012-12-12 12:23:34.56789", 2, + ClickHouseValues.UTC_TIMEZONE), ClickHouseStringValue.of("key"), ClickHouseIntegerValue.of(1), ClickHouseBigIntegerValue.of(BigInteger.TEN)); Assert.assertEquals(request.getQuery(), sql); @@ -310,8 +336,10 @@ public void testParams() { "select 1.0 as one, NULL as two, * from my_table where key=NULL and arr[NULL] in numbers(NULL)"); params.put("one", ClickHouseStringValue.of("").toSqlExpression()); - params.put("two", ClickHouseDateTimeValue.of("2012-12-12 12:23:34.56789", 2, ClickHouseValues.UTC_TIMEZONE) - .toSqlExpression()); + params.put("two", + ClickHouseDateTimeValue + .of("2012-12-12 12:23:34.56789", 2, ClickHouseValues.UTC_TIMEZONE) + .toSqlExpression()); params.put("key", ClickHouseStringValue.of("key").toSqlExpression()); params.put("some", ClickHouseBigIntegerValue.of(BigInteger.ONE).toSqlExpression()); params.put("idx", ClickHouseIntegerValue.of(1).toSqlExpression()); @@ -328,7 +356,8 @@ public void testSeal() { ClickHouseRequest request = ClickHouseClient.newInstance().connect(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])).build()); + request.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0])) + .build()); request.format(ClickHouseFormat.Avro); request.table("table1", "query_id1"); request.query("select :a", UUID.randomUUID().toString()); @@ -419,7 +448,8 @@ public void testSettings() { Assert.assertEquals(request.getStatements().get(0), "SET enable_optimize_predicate_expression=1"); request.set("log_queries_min_type", "EXCEPTION_WHILE_PROCESSING"); Assert.assertEquals(request.getStatements().size(), 2); - Assert.assertEquals(request.getStatements().get(1), "SET log_queries_min_type='EXCEPTION_WHILE_PROCESSING'"); + Assert.assertEquals(request.getStatements().get(1), + "SET log_queries_min_type='EXCEPTION_WHILE_PROCESSING'"); } @Test(groups = { "unit" }) diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseOptionTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseOptionTest.java index d61ed1ea7..ceca349db 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseOptionTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseOptionTest.java @@ -61,20 +61,23 @@ public String getDescription() { @Test(groups = { "unit" }) public void testFromString() { - Assert.assertThrows(IllegalArgumentException.class, - () -> ClickHouseOption.fromString(null, String.class)); + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseOption.fromString(null, null)); + + Assert.assertEquals(ClickHouseOption.fromString(null, String.class), ""); Assert.assertEquals(ClickHouseOption.fromString("", String.class), ""); Assert.assertEquals(ClickHouseOption.fromString("", Boolean.class), Boolean.FALSE); Assert.assertEquals(ClickHouseOption.fromString("Yes", Boolean.class), Boolean.FALSE); + Assert.assertEquals(ClickHouseOption.fromString("1", boolean.class), true); Assert.assertEquals(ClickHouseOption.fromString("1", Boolean.class), Boolean.TRUE); Assert.assertEquals(ClickHouseOption.fromString("true", Boolean.class), Boolean.TRUE); Assert.assertEquals(ClickHouseOption.fromString("True", Boolean.class), Boolean.TRUE); + Assert.assertEquals(ClickHouseOption.fromString(null, Integer.class), Integer.valueOf(0)); Assert.assertEquals(ClickHouseOption.fromString("", Integer.class), Integer.valueOf(0)); + Assert.assertEquals(ClickHouseOption.fromString(" ", 1), 1); Assert.assertEquals(ClickHouseOption.fromString("0", Integer.class), Integer.valueOf(0)); - Assert.assertThrows(IllegalArgumentException.class, - () -> ClickHouseOption.fromString(null, Integer.class)); + Assert.assertEquals(ClickHouseOption.fromString("0", int.class), 0); Assert.assertEquals(ClickHouseOption.fromString("0.1", Float.class), Float.valueOf(0.1F)); Assert.assertEquals(ClickHouseOption.fromString("NaN", Float.class), Float.valueOf(Float.NaN)); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java index d89f63181..ae4491e35 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java @@ -1,5 +1,6 @@ package com.clickhouse.jdbc; +import java.io.Serializable; import java.net.URI; import java.sql.CallableStatement; import java.sql.Connection; @@ -9,6 +10,7 @@ import java.sql.Statement; import java.util.Calendar; import java.util.Collections; +import java.util.Map; import java.util.Optional; import java.util.TimeZone; @@ -227,6 +229,15 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re */ JdbcConfig getJdbcConfig(); + /** + * Gets max insert block size. Pay attention that INSERT into one partition in + * one table of + * MergeTree family up to max_insert_size rows is transactional. + * + * @return + */ + long getMaxInsertBlockSize(); + /** * Checks whether transaction is supported. * @@ -254,6 +265,21 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re * @param sql sql to parse * @param config configuration which might be used for parsing, could be null * @return non-null parsed sql statements + * @deprecated will be removed in v0.3.3, please use + * {@link #parse(String, ClickHouseConfig, Map)} instead + */ + @Deprecated + default ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config) { + return parse(sql, config, null); + } + + /** + * Parses the given sql. + * + * @param sql sql to parse + * @param config configuration which might be used for parsing, could be null + * @param settings server settings + * @return non-null parsed sql statements */ - ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config); + ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config, Map settings); } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java index b185da033..607d0f5d7 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java @@ -12,7 +12,11 @@ public class JdbcParseHandler extends ParseHandler { private static final String SETTING_MUTATIONS_SYNC = "mutations_sync"; - public static final ParseHandler INSTANCE = new JdbcParseHandler(); + public static final ParseHandler INSTANCE = new JdbcParseHandler(false, false); + public static final ParseHandler WITHOUT_DELETE = new JdbcParseHandler(true, false); + + private final boolean allowLightWeightDelete; + private final boolean allowLightWeightUpdate; private void addMutationSetting(String sql, StringBuilder builder, Map positions, Map settings, int index) { @@ -85,15 +89,19 @@ public ClickHouseSqlStatement handleStatement(String sql, StatementType stmtType Map positions, Map settings, Set tempTables) { ClickHouseSqlStatement s = null; if (stmtType == StatementType.DELETE) { - s = handleDelete(sql, stmtType, cluster, database, table, input, format, outfile, parameters, positions, - settings, tempTables); + s = allowLightWeightDelete ? s + : handleDelete(sql, stmtType, cluster, database, table, input, format, outfile, parameters, + positions, settings, tempTables); } else if (stmtType == StatementType.UPDATE) { - s = handleUpdate(sql, stmtType, cluster, database, table, input, format, outfile, parameters, positions, - settings, tempTables); + s = allowLightWeightUpdate ? s + : handleUpdate(sql, stmtType, cluster, database, table, input, format, outfile, parameters, + positions, settings, tempTables); } return s; } - private JdbcParseHandler() { + private JdbcParseHandler(boolean allowLightWeightDelete, boolean allowLightWeightUpdate) { + this.allowLightWeightDelete = allowLightWeightDelete; + this.allowLightWeightUpdate = allowLightWeightUpdate; } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java index 78b58059d..a89a024c9 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java @@ -1,5 +1,6 @@ package com.clickhouse.jdbc.internal; +import java.io.Serializable; import java.net.URI; import java.sql.ClientInfoStatus; import java.sql.Connection; @@ -43,6 +44,7 @@ import com.clickhouse.client.ClickHouseValues; import com.clickhouse.client.ClickHouseVersion; import com.clickhouse.client.config.ClickHouseClientOption; +import com.clickhouse.client.config.ClickHouseOption; import com.clickhouse.client.config.ClickHouseRenameMethod; import com.clickhouse.client.http.config.ClickHouseHttpOption; import com.clickhouse.client.logging.Logger; @@ -59,12 +61,15 @@ import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser.ConnectionInfo; import com.clickhouse.jdbc.parser.ClickHouseSqlParser; import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; +import com.clickhouse.jdbc.parser.ParseHandler; import com.clickhouse.jdbc.parser.StatementType; public class ClickHouseConnectionImpl extends JdbcWrapper implements ClickHouseConnection { private static final Logger log = LoggerFactory.getLogger(ClickHouseConnectionImpl.class); - private static final String SETTING_READONLY = "readonly"; + static final String SETTING_READONLY = "readonly"; + static final String SETTING_MAX_INSERT_BLOCK = "max_insert_block_size"; + static final String SETTING_LW_DELETE = "allow_experimental_lightweight_delete"; private static final String SQL_GET_SERVER_INFO = "select currentUser() user, timezone() timezone, version() version, " + getSetting(SETTING_READONLY, ClickHouseDataType.UInt8) + ", " @@ -75,8 +80,8 @@ public class ClickHouseConnectionImpl extends JdbcWrapper implements ClickHouseC ClickHouseDataType.String) + "," + getSetting(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION, ClickHouseDataType.Int8) + ", " - + getSetting("max_insert_block_size", ClickHouseDataType.Int8) + ", " - + getSetting("allow_experimental_lightweight_delete", ClickHouseDataType.Int8) + + getSetting(SETTING_MAX_INSERT_BLOCK, ClickHouseDataType.UInt64) + ", " + + getSetting(SETTING_LW_DELETE, ClickHouseDataType.Int8) + " FORMAT RowBinaryWithNamesAndTypes"; private static String getSetting(String setting, ClickHouseDataType type) { @@ -157,6 +162,8 @@ protected static ClickHouseRecord getServerInfo(ClickHouseNode node, ClickHouseR private final int initialNonTxQuerySupport; private final String initialTxCommitWaitMode; private final int initialImplicitTx; + private final long initialMaxInsertBlockSize; + private final int initialDeleteSupport; private final Map> typeMap; @@ -309,15 +316,17 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { initialNonTxQuerySupport = r.getValue(4).asInteger(); initialTxCommitWaitMode = r.getValue(5).asString(); initialImplicitTx = r.getValue(6).asInteger(); + initialMaxInsertBlockSize = r.getValue(7).asLong(); + initialDeleteSupport = r.getValue(8).asInteger(); } else { - initialReadOnly = (int) clientRequest.getSettings().getOrDefault(SETTING_READONLY, 0); - initialNonTxQuerySupport = (int) clientRequest.getSettings() - .getOrDefault(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION, 1); - initialTxCommitWaitMode = (String) clientRequest.getSettings() - .getOrDefault(ClickHouseTransaction.SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE, - "wait_unknown"); - initialImplicitTx = (int) clientRequest.getSettings() - .getOrDefault(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION, 0); + initialReadOnly = clientRequest.getSetting(SETTING_READONLY, 0); + initialNonTxQuerySupport = clientRequest + .getSetting(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION, 1); + initialTxCommitWaitMode = clientRequest.getSetting( + ClickHouseTransaction.SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE, "wait_unknown"); + initialImplicitTx = clientRequest.getSetting(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION, 0); + initialMaxInsertBlockSize = clientRequest.getSetting(SETTING_MAX_INSERT_BLOCK, 0L); + initialDeleteSupport = clientRequest.getSetting(SETTING_LW_DELETE, 0); } } else { ClickHouseRecord r = getServerInfo(node, clientRequest, jdbcConf.isCreateDbIfNotExist()); @@ -339,6 +348,8 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { initialNonTxQuerySupport = r.getValue(4).asInteger(); initialTxCommitWaitMode = r.getValue(5).asString(); initialImplicitTx = r.getValue(6).asInteger(); + initialMaxInsertBlockSize = r.getValue(7).asLong(); + initialDeleteSupport = r.getValue(8).asInteger(); // update request and corresponding config clientRequest.option(ClickHouseClientOption.SERVER_TIME_ZONE, tz) @@ -349,13 +360,17 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { this.closed = false; this.database = config.getDatabase(); this.clientRequest.use(this.database); - this.readOnly = initialReadOnly != 0; + this.readOnly = clientRequest.getSetting(SETTING_READONLY, initialReadOnly) != 0; this.networkTimeout = 0; this.rsHoldability = ResultSet.HOLD_CURSORS_OVER_COMMIT; if (isTransactionSupported()) { this.txIsolation = Connection.TRANSACTION_REPEATABLE_READ; if (jdbcConf.isJdbcCompliant() && !this.readOnly) { - this.clientRequest.set(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION, 0); + if (!this.clientRequest + .hasSetting(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION)) { + this.clientRequest.set(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION, + 0); + } // .set(ClickHouseTransaction.SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE, // "wait_unknown"); } @@ -704,7 +719,7 @@ public PreparedStatement prepareStatement(String sql, int resultSetType, int res ClickHouseConfig config = clientRequest.getConfig(); // TODO remove the extra parsing - ClickHouseSqlStatement[] stmts = parse(sql, config); + ClickHouseSqlStatement[] stmts = parse(sql, config, clientRequest.getSettings()); if (stmts.length != 1) { throw SqlExceptionUtils .clientError("Prepared statement only supports one query but we got: " + stmts.length); @@ -1031,6 +1046,11 @@ public JdbcConfig getJdbcConfig() { return jdbcConf; } + @Override + public long getMaxInsertBlockSize() { + return initialMaxInsertBlockSize; + } + @Override public boolean isTransactionSupported() { return jdbcConf.isTransactionSupported() && initialNonTxQuerySupport >= 0 @@ -1050,9 +1070,20 @@ public String newQueryId() { } @Override - public ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config) { - return ClickHouseSqlParser.parse(sql, config != null ? config : clientRequest.getConfig(), - jdbcConf.isJdbcCompliant() ? JdbcParseHandler.INSTANCE : null); + public ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config, Map settings) { + ParseHandler handler = null; + if (jdbcConf.isJdbcCompliant()) { + handler = JdbcParseHandler.INSTANCE; + if (settings != null) { + Serializable value = settings.get(SETTING_LW_DELETE); + if (value == null ? initialDeleteSupport == 1 + : ClickHouseOption.fromString(value.toString(), Boolean.class)) { + handler = JdbcParseHandler.WITHOUT_DELETE; + } + + } + } + return ClickHouseSqlParser.parse(sql, config != null ? config : clientRequest.getConfig(), handler); } @Override diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java index 011482f6b..343a0d816 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java @@ -204,7 +204,7 @@ protected void setLastStatement(ClickHouseSqlStatement stmt) { } protected ClickHouseSqlStatement parseSqlStatements(String sql) { - parsedStmts = connection.parse(sql, getConfig()); + parsedStmts = connection.parse(sql, getConfig(), request.getSettings()); if (parsedStmts == null || parsedStmts.length == 0) { // should never happen @@ -567,7 +567,7 @@ public int getResultSetType() throws SQLException { public void addBatch(String sql) throws SQLException { ensureOpen(); - for (ClickHouseSqlStatement s : connection.parse(sql, getConfig())) { + for (ClickHouseSqlStatement s : connection.parse(sql, getConfig(), request.getSettings())) { this.batchStmts.add(s); } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java index c84ace208..86377c51b 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java @@ -2,10 +2,13 @@ import java.sql.SQLException; import java.sql.Savepoint; +import java.util.Properties; +import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.jdbc.ClickHouseConnection; import com.clickhouse.jdbc.ClickHouseStatement; import com.clickhouse.jdbc.JdbcIntegrationTest; +import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; import org.testng.Assert; import org.testng.annotations.Test; @@ -195,6 +198,38 @@ public void testManualRollback() throws SQLException { } } + @Test(groups = "integration") + public void testParse() throws SQLException { + Properties props = new Properties(); + String sql = "delete from table where column=1"; + boolean supportsLightWeightDelete = false; + try (ClickHouseConnection conn = newConnection(props)) { + ClickHouseSqlStatement[] stmts = conn.parse(sql, conn.getConfig(), null); + Assert.assertEquals(stmts.length, 1); + Assert.assertEquals(stmts[0].getSQL(), + "ALTER TABLE `table` DELETE where column=1 SETTINGS mutations_sync=1"); + if (conn.getServerVersion().check("[22.8,)")) { + supportsLightWeightDelete = true; + } + } + + if (!supportsLightWeightDelete) { + return; + } + + props.setProperty("custom_settings", "allow_experimental_lightweight_delete=1"); + try (ClickHouseConnection conn = newConnection(props)) { + ClickHouseSqlStatement[] stmts = conn.parse(sql, conn.getConfig(), null); + Assert.assertEquals(stmts.length, 1); + Assert.assertEquals(stmts[0].getSQL(), + "ALTER TABLE `table` DELETE where column=1 SETTINGS mutations_sync=1"); + + stmts = conn.parse(sql, conn.getConfig(), conn.unwrap(ClickHouseRequest.class).getSettings()); + Assert.assertEquals(stmts.length, 1); + Assert.assertEquals(stmts[0].getSQL(), sql); + } + } + @Test(groups = "integration") public void testSwitchAutoCommit() throws SQLException { try (ClickHouseConnection conn = newConnection()) {