diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java index 55eec3345..743bf5678 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java @@ -19,6 +19,7 @@ import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseDataType; +import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseParameterizedQuery; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValues; @@ -66,9 +67,11 @@ protected ResultSet query(String sql, ClickHouseRecordTransformer func, boolean SQLException error = null; try { ClickHouseStatement stmt = connection.createStatement(); + stmt.setLargeMaxRows(0L); return new ClickHouseResultSet("", "", stmt, // load everything into memory ClickHouseSimpleResponse.of(stmt.getRequest() + .format(ClickHouseFormat.RowBinaryWithNamesAndTypes) .option(ClickHouseClientOption.RENAME_RESPONSE_COLUMN, ClickHouseRenameMethod.NONE) .query(sql).execute().get(), func)); } catch (InterruptedException e) { 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 94de8e5ec..1b0be66e0 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 @@ -388,10 +388,10 @@ public void setLargeMaxRows(long max) throws SQLException { if (this.maxRows != max) { if (max == 0L || !connection.allowCustomSetting()) { - request.removeSetting("max_result_rows"); + request.removeSetting(ClickHouseClientOption.MAX_RESULT_ROWS.getKey()); request.removeSetting("result_overflow_mode"); } else { - request.set("max_result_rows", max); + request.set(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), max); request.set("result_overflow_mode", "break"); } this.maxRows = max; diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaDataTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaDataTest.java index dccdeb625..31be07d65 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaDataTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaDataTest.java @@ -102,6 +102,21 @@ public void testGetColumns(String columnType, Integer columnSize, Integer decima } } + @Test(groups = "integration") + public void testMaxRows() throws SQLException { + Properties props = new Properties(); + props.setProperty(ClickHouseClientOption.MAX_RESULT_ROWS.getKey(), "1"); + int count = 0; + try (ClickHouseConnection conn = newConnection(props)) { + try (ResultSet rs = conn.getMetaData().getColumns(conn.getCatalog(), conn.getSchema(), "%", "%")) { + while (rs.next()) { + count++; + } + } + } + Assert.assertTrue(count > 1, "Should have more than one row returned"); + } + @Test(groups = "integration") public void testTableComment() throws SQLException { String tableName = "test_table_comment";