Skip to content

Commit 537663b

Browse files
committed
Return read/written rows whenever possible
1 parent cdaf8f3 commit 537663b

File tree

4 files changed

+53
-42
lines changed

4 files changed

+53
-42
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponseSummary.java

+30-5
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,8 @@
1010
public class ClickHouseResponseSummary implements Serializable {
1111
private static final long serialVersionUID = 6241261266635143197L;
1212

13+
static final String ERROR_CANNOT_UPDATE = "Sealed summary cannot be updated";
14+
1315
public static final ClickHouseResponseSummary EMPTY = new ClickHouseResponseSummary(null, null, true);
1416

1517
/**
@@ -61,6 +63,11 @@ public long getWrittenRows() {
6163
public long getWrittenBytes() {
6264
return written_bytes;
6365
}
66+
67+
public boolean isEmpty() {
68+
return read_rows == 0L && read_bytes == 0L && total_rows_to_read == 0L && written_rows == 0L
69+
&& written_bytes == 0L;
70+
}
6471
}
6572

6673
/**
@@ -112,6 +119,10 @@ public boolean hasAppliedLimit() {
112119
public long getRowsBeforeLimit() {
113120
return rows_before_limit;
114121
}
122+
123+
public boolean isEmpty() {
124+
return rows == 0L && blocks == 0L && allocated_bytes == 0L && !applied_limit && rows_before_limit == 0L;
125+
}
115126
}
116127

117128
private final AtomicReference<Progress> progress;
@@ -139,9 +150,15 @@ public ClickHouseResponseSummary(Progress progress, Statistics stats) {
139150
* @param sealed whether the summary is sealed
140151
*/
141152
protected ClickHouseResponseSummary(Progress progress, Statistics stats, boolean sealed) {
142-
this.progress = new AtomicReference<>(progress != null ? progress : new Progress(0L, 0L, 0L, 0L, 0L));
143-
this.stats = new AtomicReference<>(stats != null ? stats : new Statistics(0L, 0L, 0L, false, 0L));
144-
this.updates = new AtomicInteger(1);
153+
if (progress == null) {
154+
progress = new Progress(0L, 0L, 0L, 0L, 0L);
155+
}
156+
if (stats == null) {
157+
stats = new Statistics(0L, 0L, 0L, false, 0L);
158+
}
159+
this.progress = new AtomicReference<>(progress);
160+
this.stats = new AtomicReference<>(stats);
161+
this.updates = new AtomicInteger(progress.isEmpty() && stats.isEmpty() ? 0 : 1);
145162

146163
this.sealed = sealed;
147164
}
@@ -159,6 +176,10 @@ public void seal() {
159176
* @return increased update counter
160177
*/
161178
public int update() {
179+
if (sealed) {
180+
throw new IllegalStateException(ERROR_CANNOT_UPDATE);
181+
}
182+
162183
return this.updates.incrementAndGet();
163184
}
164185

@@ -169,7 +190,7 @@ public int update() {
169190
*/
170191
public void update(Progress progress) {
171192
if (sealed) {
172-
throw new IllegalStateException("Sealed summary cannot be updated");
193+
throw new IllegalStateException(ERROR_CANNOT_UPDATE);
173194
}
174195

175196
if (progress != null) {
@@ -179,7 +200,7 @@ public void update(Progress progress) {
179200

180201
public void update(Statistics stats) {
181202
if (sealed) {
182-
throw new IllegalStateException("Sealed summary cannot be updated");
203+
throw new IllegalStateException(ERROR_CANNOT_UPDATE);
183204
}
184205

185206
if (stats != null) {
@@ -228,4 +249,8 @@ public long getWrittenBytes() {
228249
public int getUpdateCount() {
229250
return updates.get();
230251
}
252+
253+
public boolean isEmpty() {
254+
return progress.get().isEmpty() && stats.get().isEmpty();
255+
}
231256
}

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

+15-30
Original file line numberDiff line numberDiff line change
@@ -95,8 +95,13 @@ private ClickHouseResponse getLastResponse(Map<ClickHouseOption, Serializable> o
9595
} catch (Exception e) {
9696
throw SqlExceptionUtils.handle(e);
9797
} finally {
98-
if (i + 1 < len && response != null) {
98+
if (response == null) {
99+
// something went wrong
100+
} else if (i + 1 < len) {
99101
response.close();
102+
response = null;
103+
} else {
104+
updateResult(stmt, response);
100105
}
101106
}
102107
}
@@ -166,18 +171,17 @@ protected ClickHouseResponse executeStatement(ClickHouseSqlStatement stmt,
166171

167172
protected int executeInsert(String sql, InputStream input) throws SQLException {
168173
boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported();
169-
ClickHouseResponseSummary summary = null;
170174
Mutation req = request.write().query(sql, queryId = connection.newQueryId()).data(input);
171175
try (ClickHouseResponse resp = autoTx
172176
? req.executeWithinTransaction(connection.isImplicitTransactionSupported())
173177
: req.transaction(connection.getTransaction()).sendAndWait();
174178
ResultSet rs = updateResult(new ClickHouseSqlStatement(sql, StatementType.INSERT), resp)) {
175-
summary = resp.getSummary();
179+
// ignore
176180
} catch (Exception e) {
177181
throw SqlExceptionUtils.handle(e);
178182
}
179183

180-
return summary != null && summary.getWrittenRows() > 0L ? (int) summary.getWrittenRows() : 1;
184+
return (int) currentUpdateCount;
181185
}
182186

183187
protected ClickHouseSqlStatement getLastStatement() {
@@ -212,23 +216,17 @@ protected ClickHouseResultSet newEmptyResultSet() throws SQLException {
212216
}
213217

214218
protected ResultSet updateResult(ClickHouseSqlStatement stmt, ClickHouseResponse response) throws SQLException {
215-
ResultSet rs = null;
216219
if (stmt.isQuery() || !response.getColumns().isEmpty()) {
217220
currentUpdateCount = -1L;
218221
currentResult = new ClickHouseResultSet(stmt.getDatabaseOrDefault(getConnection().getCurrentDatabase()),
219222
stmt.getTable(), this, response);
220-
rs = currentResult;
221223
} else {
222-
currentUpdateCount = response.getSummary().getWrittenRows();
223-
// FIXME apparently this is not always true
224-
if (currentUpdateCount <= 0L) {
225-
currentUpdateCount = 1L;
226-
}
227-
currentResult = null;
228224
response.close();
225+
currentUpdateCount = stmt.isDDL() ? 0L
226+
: (response.getSummary().isEmpty() ? 1L : response.getSummary().getWrittenRows());
227+
currentResult = null;
229228
}
230-
231-
return rs == null ? newEmptyResultSet() : rs;
229+
return currentResult;
232230
}
233231

234232
protected ClickHouseStatementImpl(ClickHouseConnectionImpl connection, ClickHouseRequest<?> request,
@@ -303,18 +301,8 @@ public ResultSet executeQuery(String sql) throws SQLException {
303301
}
304302

305303
parseSqlStatements(sql);
306-
307-
ClickHouseResponse response = getLastResponse(null, null, null);
308-
309-
try {
310-
return updateResult(getLastStatement(), response);
311-
} catch (Exception e) {
312-
if (response != null) {
313-
response.close();
314-
}
315-
316-
throw SqlExceptionUtils.handle(e);
317-
}
304+
getLastResponse(null, null, null);
305+
return currentResult != null ? currentResult : newEmptyResultSet();
318306
}
319307

320308
@Override
@@ -326,14 +314,11 @@ public long executeLargeUpdate(String sql) throws SQLException {
326314

327315
parseSqlStatements(sql);
328316

329-
ClickHouseResponseSummary summary = null;
330317
try (ClickHouseResponse response = getLastResponse(null, null, null)) {
331-
summary = response.getSummary();
318+
return currentUpdateCount;
332319
} catch (Exception e) {
333320
throw SqlExceptionUtils.handle(e);
334321
}
335-
336-
return summary != null ? summary.getWrittenRows() : 1L;
337322
}
338323

339324
@Override

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

+5-4
Original file line numberDiff line numberDiff line change
@@ -845,6 +845,7 @@ public void testBatchQuery() throws SQLException {
845845
@Test(dataProvider = "statementAndParams", groups = "integration")
846846
public void testExecuteWithOrWithoutParameters(String tableSuffix, String query, Class<?> clazz,
847847
boolean hasResultSet, String[] params, boolean checkTable) throws SQLException {
848+
int expectedRowCount = "ddl".equals(tableSuffix) ? 0 : 1;
848849
String tableName = "test_execute_ps_" + tableSuffix;
849850
query = query.replace("$table", tableName);
850851
Properties props = new Properties();
@@ -932,7 +933,7 @@ public void testExecuteWithOrWithoutParameters(String tableSuffix, String query,
932933
if (hasResultSet) {
933934
Assert.assertThrows(SQLException.class, () -> ps.executeLargeBatch());
934935
} else {
935-
Assert.assertEquals(ps.executeLargeBatch(), new long[] { 1L });
936+
Assert.assertEquals(ps.executeLargeBatch(), new long[] { expectedRowCount });
936937
}
937938
if (checkTable)
938939
checkTable(stmt, "select * from " + tableName, params);
@@ -950,7 +951,7 @@ public void testExecuteWithOrWithoutParameters(String tableSuffix, String query,
950951
if (hasResultSet) {
951952
Assert.assertThrows(SQLException.class, () -> ps.executeBatch());
952953
} else {
953-
Assert.assertEquals(ps.executeBatch(), new int[] { 1 });
954+
Assert.assertEquals(ps.executeBatch(), new int[] { expectedRowCount });
954955
}
955956
if (checkTable)
956957
checkTable(stmt, "select * from " + tableName, params);
@@ -973,7 +974,7 @@ public void testExecuteWithOrWithoutParameters(String tableSuffix, String query,
973974
if (hasResultSet) {
974975
Assert.assertEquals(ps.executeLargeBatch(), new long[] { Statement.EXECUTE_FAILED });
975976
} else {
976-
Assert.assertEquals(ps.executeLargeBatch(), new long[] { 1L });
977+
Assert.assertEquals(ps.executeLargeBatch(), new long[] { expectedRowCount });
977978
}
978979
if (checkTable)
979980
checkTable(stmt, "select * from " + tableName, params);
@@ -988,7 +989,7 @@ public void testExecuteWithOrWithoutParameters(String tableSuffix, String query,
988989
if (hasResultSet) {
989990
Assert.assertEquals(ps.executeBatch(), new int[] { Statement.EXECUTE_FAILED });
990991
} else {
991-
Assert.assertEquals(ps.executeBatch(), new int[] { 1 });
992+
Assert.assertEquals(ps.executeBatch(), new int[] { expectedRowCount });
992993
}
993994
if (checkTable)
994995
checkTable(stmt, "select * from " + tableName, params);

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

+3-3
Original file line numberDiff line numberDiff line change
@@ -173,7 +173,7 @@ public void testMutation() throws SQLException {
173173
// [update] tbl a [set] a.b = 1 where a.b != 1[ settings mutation_async=0]
174174
// alter table tbl a update a.b = 1 where a.b != 1
175175
conn.setClientInfo("ApplicationName", "333");
176-
Assert.assertEquals(conn.createStatement().executeUpdate("update test_mutation set b = 22 where b = 1"), 0);
176+
Assert.assertEquals(conn.createStatement().executeUpdate("update test_mutation set b = 22 where b = 1"), 1);
177177

178178
Assert.assertThrows(SQLException.class,
179179
() -> stmt.executeUpdate("update non_existing_table set value=1 where key=1"));
@@ -407,7 +407,7 @@ public void testExecuteQuery() throws SQLException {
407407
rs = stmt.executeQuery("drop table if exists non_existing_table");
408408
Assert.assertNotNull(rs, "Should never be null");
409409
Assert.assertNull(stmt.getResultSet(), "Should be null");
410-
Assert.assertEquals(stmt.getUpdateCount(), 1);
410+
Assert.assertEquals(stmt.getUpdateCount(), 0);
411411
Assert.assertFalse(rs.next(), "Should has no row");
412412
}
413413
}
@@ -519,7 +519,7 @@ public void testQuerySystemLog() throws SQLException {
519519
stmt.addBatch("drop table if exists non_existing_table2");
520520
stmt.addBatch("drop table if exists non_existing_table3");
521521
int[] results = stmt.executeBatch();
522-
Assert.assertEquals(results, new int[] { 1, 1, 1 });
522+
Assert.assertEquals(results, new int[] { 0, 0, 0 });
523523
}
524524
}
525525

0 commit comments

Comments
 (0)