33import com .clickhouse .client .api .ClientConfigProperties ;
44import com .clickhouse .client .api .data_formats .ClickHouseBinaryFormatReader ;
55import com .clickhouse .client .api .internal .ServerSettings ;
6- import com .clickhouse .client .api .metrics .OperationMetrics ;
7- import com .clickhouse .client .api .metrics .ServerMetrics ;
86import com .clickhouse .client .api .query .QueryResponse ;
97import com .clickhouse .client .api .query .QuerySettings ;
108import com .clickhouse .client .api .sql .SQLUtils ;
1614
1715import java .sql .ResultSet ;
1816import java .sql .SQLException ;
19- import java .sql .SQLFeatureNotSupportedException ;
2017import java .sql .SQLWarning ;
2118import java .sql .Statement ;
2219import java .util .ArrayList ;
@@ -35,9 +32,9 @@ public class StatementImpl implements Statement, JdbcV2Wrapper {
3532
3633 // State
3734 private volatile boolean closed ;
38- private final ConcurrentLinkedQueue <ResultSetImpl > resultSets ;
35+ private final ConcurrentLinkedQueue <ResultSetImpl > resultSets ; // all result sets linked to this statement
3936 protected ResultSetImpl currentResultSet ;
40- protected OperationMetrics metrics ;
37+ protected long currentUpdateCount = - 1 ;
4138 protected List <String > batch ;
4239 private String lastStatementSql ;
4340 private ParsedStatement parsedStatement ;
@@ -54,7 +51,6 @@ public StatementImpl(ConnectionImpl connection) throws SQLException {
5451 this .connection = connection ;
5552 this .queryTimeout = 0 ;
5653 this .closed = false ;
57- this .metrics = null ;
5854 this .batch = new ArrayList <>();
5955 this .maxRows = 0 ;
6056 this .localSettings = QuerySettings .merge (connection .getDefaultQuerySettings (), new QuerySettings ());
@@ -97,7 +93,9 @@ protected String getLastStatementSql() {
9793 @ Override
9894 public ResultSet executeQuery (String sql ) throws SQLException {
9995 ensureOpen ();
100- return executeQueryImpl (sql , localSettings );
96+ currentUpdateCount = -1 ;
97+ currentResultSet = executeQueryImpl (sql , localSettings );
98+ return currentResultSet ;
10199 }
102100
103101 private void closeCurrentResultSet () {
@@ -149,9 +147,7 @@ protected ResultSetImpl executeQueryImpl(String sql, QuerySettings settings) thr
149147 if (reader .getSchema () == null ) {
150148 throw new SQLException ("Called method expects empty or filled result set but query has returned none. Consider using `java.sql.Statement.execute(java.lang.String)`" , ExceptionUtils .SQL_STATE_CLIENT_ERROR );
151149 }
152- metrics = response .getMetrics ();
153- setCurrentResultSet (new ResultSetImpl (this , response , reader ));
154- return currentResultSet ;
150+ return new ResultSetImpl (this , response , reader );
155151 } catch (Exception e ) {
156152 if (response != null ) {
157153 try {
@@ -168,13 +164,10 @@ protected ResultSetImpl executeQueryImpl(String sql, QuerySettings settings) thr
168164 @ Override
169165 public int executeUpdate (String sql ) throws SQLException {
170166 ensureOpen ();
171- parsedStatement = connection .getSqlParser ().parsedStatement (sql );
172- int updateCount = executeUpdateImpl (sql , localSettings );
173- postUpdateActions ();
174- return updateCount ;
167+ return (int )executeLargeUpdate (sql );
175168 }
176169
177- protected int executeUpdateImpl (String sql , QuerySettings settings ) throws SQLException {
170+ protected long executeUpdateImpl (String sql , QuerySettings settings ) throws SQLException {
178171 ensureOpen ();
179172
180173 // Closing before trying to do next request. Otherwise, deadlock because previous connection will not be
@@ -196,9 +189,7 @@ protected int executeUpdateImpl(String sql, QuerySettings settings) throws SQLEx
196189 int updateCount = 0 ;
197190 try (QueryResponse response = queryTimeout == 0 ? connection .client .query (lastStatementSql , mergedSettings ).get ()
198191 : connection .client .query (lastStatementSql , mergedSettings ).get (queryTimeout , TimeUnit .SECONDS )) {
199- setCurrentResultSet (null );
200192 updateCount = Math .max (0 , (int ) response .getWrittenRows ()); // when statement alters schema no result rows returned.
201- metrics = response .getMetrics ();
202193 lastQueryId = response .getQueryId ();
203194 } catch (Exception e ) {
204195 throw ExceptionUtils .toSqlState (e );
@@ -308,28 +299,17 @@ public void setCursorName(String name) throws SQLException {
308299 ensureOpen ();
309300 }
310301
311- /**
312- * Remembers current result set to be able to close it later.
313- * Sets current resultset to a new value
314- * @param resultSet new current resultset
315- */
316- protected void setCurrentResultSet (ResultSetImpl resultSet ) {
317- ResultSetImpl tmp = currentResultSet ;
318- currentResultSet = resultSet ;
319- if (tmp != null ) {
320- resultSets .add (tmp );
321- }
322- }
323-
324302 @ Override
325303 public boolean execute (String sql ) throws SQLException {
326304 ensureOpen ();
327305 parsedStatement = connection .getSqlParser ().parsedStatement (sql );
306+ currentUpdateCount = -1 ;
307+ currentResultSet = null ;
328308 if (parsedStatement .isHasResultSet ()) {
329- executeQueryImpl (sql , localSettings ); // keep open to allow getResultSet()
309+ currentResultSet = executeQueryImpl (sql , localSettings );
330310 return true ;
331311 } else {
332- executeUpdateImpl (sql , localSettings );
312+ currentUpdateCount = executeUpdateImpl (sql , localSettings );
333313 postUpdateActions ();
334314 return false ;
335315 }
@@ -339,9 +319,7 @@ public boolean execute(String sql) throws SQLException {
339319 public ResultSet getResultSet () throws SQLException {
340320 ensureOpen ();
341321
342- ResultSet resultSet = currentResultSet ;
343- setCurrentResultSet (null );
344- return resultSet ;
322+ return currentResultSet ;
345323 }
346324
347325 @ Override
@@ -353,7 +331,7 @@ public int getUpdateCount() throws SQLException {
353331 @ Override
354332 public boolean getMoreResults () throws SQLException {
355333 ensureOpen ();
356- return false ;
334+ return getMoreResults ( Statement . CLOSE_CURRENT_RESULT ) ;
357335 }
358336
359337 @ Override
@@ -435,8 +413,17 @@ public QuerySettings getLocalSettings() {
435413
436414 @ Override
437415 public boolean getMoreResults (int current ) throws SQLException {
438- // TODO: implement query batches. When multiple selects in the batch.
439- return false ;
416+ // This method designed to iterate over multiple resultsets after "execute(sql)" method is called
417+ // But we have at most only one always
418+ // Then we should close any existing and return false to indicate that no more result are present
419+
420+ if (currentResultSet != null && current != Statement .KEEP_CURRENT_RESULT ) {
421+ currentResultSet .close ();
422+ }
423+
424+ currentResultSet = null ;
425+ currentUpdateCount = -1 ;
426+ return false ; // false indicates that no more results (or it is an update count)
440427 }
441428
442429 @ Override
@@ -547,13 +534,7 @@ public boolean isCloseOnCompletion() throws SQLException {
547534 @ Override
548535 public long getLargeUpdateCount () throws SQLException {
549536 ensureOpen ();
550- if (currentResultSet == null && metrics != null ) {
551- long updateCount = metrics .getMetric (ServerMetrics .NUM_ROWS_WRITTEN ).getLong ();
552- metrics = null ;// clear metrics
553- return updateCount ;
554- }
555-
556- return -1L ;
537+ return currentUpdateCount ;
557538 }
558539
559540 @ Override
@@ -590,22 +571,25 @@ public long[] executeLargeBatch() throws SQLException {
590571
591572 @ Override
592573 public long executeLargeUpdate (String sql ) throws SQLException {
593- return executeUpdate (sql );
574+ parsedStatement = connection .getSqlParser ().parsedStatement (sql );
575+ long updateCount = executeUpdateImpl (sql , localSettings );
576+ postUpdateActions ();
577+ return updateCount ;
594578 }
595579
596580 @ Override
597581 public long executeLargeUpdate (String sql , int autoGeneratedKeys ) throws SQLException {
598- return executeUpdate (sql , autoGeneratedKeys );
582+ return executeLargeUpdate (sql );
599583 }
600584
601585 @ Override
602586 public long executeLargeUpdate (String sql , int [] columnIndexes ) throws SQLException {
603- return executeUpdate (sql , columnIndexes );
587+ return executeLargeUpdate (sql );
604588 }
605589
606590 @ Override
607591 public long executeLargeUpdate (String sql , String [] columnNames ) throws SQLException {
608- return executeUpdate (sql , columnNames );
592+ return executeLargeUpdate (sql );
609593 }
610594
611595 /**
0 commit comments