Skip to content

Commit 81f7268

Browse files
authored
Merge pull request #1934 from ClickHouse/jdbc_fix_result_overflow_mode
Fixed overriding max_result_rows settings
2 parents 90087f7 + 8c4fd73 commit 81f7268

File tree

4 files changed

+57
-10
lines changed

4 files changed

+57
-10
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -218,6 +218,8 @@ public enum ClickHouseClientOption implements ClickHouseOption {
218218
MAX_RESULT_ROWS("max_result_rows", 0L,
219219
"Limit on the number of rows in the result. "
220220
+ "Also checked for subqueries, and on remote servers when running parts of a distributed query."),
221+
222+
RESULT_OVERFLOW_MODE("result_overflow_mode", "throw","What to do if the result is overflowed."),
221223
/**
222224
* Maximum size of thread pool for each client.
223225
*/

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -118,10 +118,25 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
118118
appendQueryParameter(builder, settingKey, String.valueOf(config.getMaxExecutionTime()));
119119
}
120120
settingKey = ClickHouseClientOption.MAX_RESULT_ROWS.getKey();
121-
if (config.getMaxResultRows() > 0L && !settings.containsKey(settingKey)) {
121+
boolean hasRequestSetting = settings.containsKey(settingKey);
122+
if (config.getMaxResultRows() > 0L && !hasRequestSetting) {
123+
// set on client level
122124
appendQueryParameter(builder, settingKey, String.valueOf(config.getMaxResultRows()));
123-
appendQueryParameter(builder, "result_overflow_mode", "break");
125+
} else if (hasRequestSetting) {
126+
// set on request level
127+
Object value = settings.get(settingKey);
128+
if (value instanceof Number && ((Number) value).longValue() > 0L) {
129+
appendQueryParameter(builder, settingKey, String.valueOf(value));
130+
} else if (value instanceof String && !(((String) value).isEmpty() || "0".equals(value))) {
131+
appendQueryParameter(builder, settingKey, (String) value);
132+
}
133+
}
134+
135+
if (config.hasOption(ClickHouseClientOption.RESULT_OVERFLOW_MODE)) {
136+
appendQueryParameter(builder, ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey(),
137+
config.getStrOption(ClickHouseClientOption.RESULT_OVERFLOW_MODE));
124138
}
139+
125140
settingKey = "log_comment";
126141
if (!stmts.isEmpty() && config.getBoolOption(ClickHouseClientOption.LOG_LEADING_COMMENT)
127142
&& !settings.containsKey(settingKey)) {
@@ -162,11 +177,9 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
162177
}
163178

164179
for (Entry<String, Serializable> entry : settings.entrySet()) {
165-
// Skip internal settings
166-
if (entry.getKey().equalsIgnoreCase("_set_roles_stmt")) {
167-
continue;
180+
if (!processedSettings.contains(entry.getKey())) {
181+
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
168182
}
169-
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
170183
}
171184

172185
if (builder.length() > 0) {
@@ -175,6 +188,12 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
175188
return builder.toString();
176189
}
177190

191+
// Settings that are processed by the client and appropriate parameters are set
192+
private static final HashSet<String> processedSettings = new HashSet<>(Arrays.asList(
193+
"_set_roles_stmt",
194+
ClickHouseClientOption.MAX_RESULT_ROWS.getKey()
195+
));
196+
178197
static String buildUrl(String baseUrl, ClickHouseRequest<?> request, Map<String, Serializable> additionalParams) {
179198
StringBuilder builder = new StringBuilder().append(baseUrl);
180199
// TODO: Using default until we will remove

clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -610,11 +610,9 @@ public void setLargeMaxRows(long max) throws SQLException {
610610

611611
if (this.maxRows != max) {
612612
if (max == 0L || !connection.allowCustomSetting()) {
613-
request.removeSetting(ClickHouseClientOption.MAX_RESULT_ROWS.getKey());
614-
request.removeSetting("result_overflow_mode");
613+
request.set(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), 0);
615614
} else {
616615
request.set(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), max);
617-
request.set("result_overflow_mode", "break");
618616
}
619617
this.maxRows = max;
620618
}

clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java

Lines changed: 29 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -834,7 +834,9 @@ public void testWrapperObject() throws SQLException {
834834

835835
@Test(groups = "integration")
836836
public void testQuerySystemLog() throws SQLException {
837-
try (ClickHouseConnection conn = newConnection(new Properties())) {
837+
Properties props = new Properties();
838+
props.setProperty(ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey(), "break");
839+
try (ClickHouseConnection conn = newConnection(props)) {
838840
ClickHouseStatement stmt = conn.createStatement();
839841
stmt.setMaxRows(10);
840842
stmt.setLargeMaxRows(11L);
@@ -1493,4 +1495,30 @@ public void testDescMetadata() {
14931495
Assert.fail("Failed to create connection", e);
14941496
}
14951497
}
1498+
1499+
@Test(groups = "integration")
1500+
public void testMaxResultsRows() throws SQLException {
1501+
Properties props = new Properties();
1502+
int maxRows = 3;
1503+
props.setProperty(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), String.valueOf(maxRows));
1504+
props.setProperty(ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey(), "break");
1505+
try (ClickHouseConnection conn = newConnection(props);
1506+
ClickHouseStatement s = conn.createStatement()) {
1507+
ResultSet rs = s.executeQuery("SELECT number FROM system.numbers");
1508+
for (int i = 0; i < maxRows; i++) {
1509+
Assert.assertTrue(rs.next(), "Should have more rows, but have only " + i);
1510+
}
1511+
}
1512+
1513+
props.setProperty(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), "1");
1514+
props.remove(ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey());
1515+
try (ClickHouseConnection conn = newConnection(props);
1516+
ClickHouseStatement s = conn.createStatement()) {
1517+
s.executeQuery("SELECT number FROM system.numbers");
1518+
Assert.fail("Should throw exception");
1519+
} catch (SQLException e) {
1520+
Assert.assertTrue(e.getMessage().startsWith("Code: 396. DB::Exception: Limit for result exceeded, max rows"),
1521+
"Unexpected exception: " + e.getMessage());
1522+
}
1523+
}
14961524
}

0 commit comments

Comments
 (0)