Skip to content

Commit af0454b

Browse files
committed
Fixed overriding max_result_rows settings
1 parent 1baf186 commit af0454b

File tree

4 files changed

+52
-9
lines changed

4 files changed

+52
-9
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: 23 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -118,10 +118,23 @@ 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+
Number value = (Number) settings.get(settingKey);
128+
if (value.longValue() > 0L) {
129+
appendQueryParameter(builder, settingKey, String.valueOf(value.longValue()));
130+
}
131+
}
132+
133+
if (config.hasOption(ClickHouseClientOption.RESULT_OVERFLOW_MODE)) {
134+
appendQueryParameter(builder, ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey(),
135+
config.getStrOption(ClickHouseClientOption.RESULT_OVERFLOW_MODE));
124136
}
137+
125138
settingKey = "log_comment";
126139
if (!stmts.isEmpty() && config.getBoolOption(ClickHouseClientOption.LOG_LEADING_COMMENT)
127140
&& !settings.containsKey(settingKey)) {
@@ -162,11 +175,9 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
162175
}
163176

164177
for (Entry<String, Serializable> entry : settings.entrySet()) {
165-
// Skip internal settings
166-
if (entry.getKey().equalsIgnoreCase("_set_roles_stmt")) {
167-
continue;
178+
if (!processedSettings.contains(entry.getKey())) {
179+
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
168180
}
169-
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
170181
}
171182

172183
if (builder.length() > 0) {
@@ -175,6 +186,12 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
175186
return builder.toString();
176187
}
177188

189+
// Settings that are processed by the client and appropriate parameters are set
190+
private static final HashSet<String> processedSettings = new HashSet<>(Arrays.asList(
191+
"_set_roles_stmt",
192+
ClickHouseClientOption.MAX_RESULT_ROWS.getKey()
193+
));
194+
178195
static String buildUrl(String baseUrl, ClickHouseRequest<?> request, Map<String, Serializable> additionalParams) {
179196
StringBuilder builder = new StringBuilder().append(baseUrl);
180197
// 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: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1493,4 +1493,30 @@ public void testDescMetadata() {
14931493
Assert.fail("Failed to create connection", e);
14941494
}
14951495
}
1496+
1497+
@Test(groups = "integration")
1498+
public void testMaxResultsRows() throws SQLException {
1499+
Properties props = new Properties();
1500+
int maxRows = 3;
1501+
props.setProperty(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), String.valueOf(maxRows));
1502+
props.setProperty(ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey(), "break");
1503+
try (ClickHouseConnection conn = newConnection(props);
1504+
ClickHouseStatement s = conn.createStatement()) {
1505+
ResultSet rs = s.executeQuery("SELECT number FROM system.numbers");
1506+
for (int i = 0; i < maxRows; i++) {
1507+
Assert.assertTrue(rs.next(), "Should have more rows, but have only " + i);
1508+
}
1509+
}
1510+
1511+
props.setProperty(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), "1");
1512+
props.remove(ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey());
1513+
try (ClickHouseConnection conn = newConnection(props);
1514+
ClickHouseStatement s = conn.createStatement()) {
1515+
s.executeQuery("SELECT number FROM system.numbers");
1516+
Assert.fail("Should throw exception");
1517+
} catch (SQLException e) {
1518+
Assert.assertTrue(e.getMessage().startsWith("Code: 396. DB::Exception: Limit for result exceeded, max rows"),
1519+
"Unexpected exception: " + e.getMessage());
1520+
}
1521+
}
14961522
}

0 commit comments

Comments
 (0)