11package com .clickhouse .jdbc .internal ;
22
3- import java .io .IOException ;
4- import java .io .InputStream ;
5- import java .io .OutputStream ;
6- import java .io .Serializable ;
7- import java .nio .file .Path ;
8- import java .sql .ResultSet ;
9- import java .sql .SQLException ;
10- import java .sql .SQLFeatureNotSupportedException ;
11- import java .sql .SQLWarning ;
12- import java .sql .Statement ;
13- import java .util .*;
14- import java .util .Map .Entry ;
15- import java .util .concurrent .TimeUnit ;
16- import java .util .concurrent .TimeoutException ;
17-
183import com .clickhouse .client .ClickHouseClient ;
194import com .clickhouse .client .ClickHouseConfig ;
205import com .clickhouse .client .ClickHouseException ;
216import com .clickhouse .client .ClickHouseNode ;
227import com .clickhouse .client .ClickHouseRequest ;
8+ import com .clickhouse .client .ClickHouseRequest .Mutation ;
239import com .clickhouse .client .ClickHouseResponse ;
2410import com .clickhouse .client .ClickHouseResponseSummary ;
2511import com .clickhouse .client .ClickHouseSimpleResponse ;
2612import com .clickhouse .client .ClickHouseTransaction ;
27- import com .clickhouse .client .ClickHouseRequest .Mutation ;
2813import com .clickhouse .client .config .ClickHouseClientOption ;
2914import com .clickhouse .client .config .ClickHouseDefaults ;
3015import com .clickhouse .config .ClickHouseConfigChangeListener ;
4227import com .clickhouse .data .ClickHouseOutputStream ;
4328import com .clickhouse .data .ClickHouseUtils ;
4429import com .clickhouse .data .ClickHouseValues ;
45- import com .clickhouse .logging .Logger ;
46- import com .clickhouse .logging .LoggerFactory ;
4730import com .clickhouse .jdbc .ClickHouseConnection ;
4831import com .clickhouse .jdbc .ClickHouseResultSet ;
4932import com .clickhouse .jdbc .ClickHouseStatement ;
5033import com .clickhouse .jdbc .JdbcTypeMapping ;
51- import com .clickhouse .jdbc .SqlExceptionUtils ;
5234import com .clickhouse .jdbc .JdbcWrapper ;
35+ import com .clickhouse .jdbc .SqlExceptionUtils ;
5336import com .clickhouse .jdbc .parser .ClickHouseSqlStatement ;
5437import com .clickhouse .jdbc .parser .StatementType ;
38+ import com .clickhouse .logging .Logger ;
39+ import com .clickhouse .logging .LoggerFactory ;
40+
41+ import java .io .IOException ;
42+ import java .io .InputStream ;
43+ import java .io .OutputStream ;
44+ import java .io .Serializable ;
45+ import java .nio .file .Path ;
46+ import java .sql .ResultSet ;
47+ import java .sql .SQLException ;
48+ import java .sql .SQLFeatureNotSupportedException ;
49+ import java .sql .SQLWarning ;
50+ import java .sql .Statement ;
51+ import java .util .ArrayList ;
52+ import java .util .Arrays ;
53+ import java .util .HashSet ;
54+ import java .util .LinkedList ;
55+ import java .util .List ;
56+ import java .util .Map ;
57+ import java .util .Map .Entry ;
58+ import java .util .concurrent .TimeUnit ;
59+ import java .util .concurrent .TimeoutException ;
60+ import java .util .function .Function ;
5561
5662public class ClickHouseStatementImpl extends JdbcWrapper
5763 implements ClickHouseConfigChangeListener <ClickHouseRequest <?>>, ClickHouseStatement {
@@ -130,11 +136,9 @@ private ClickHouseResponse getLastResponse(Map<ClickHouseOption, Serializable> o
130136 request .set ("_set_roles_stmt" , requestRoles );
131137 }
132138
133- request .query (stmt .getSQL (), queryId = connection .newQueryId ());
134139 // TODO skip useless queries to reduce network calls and server load
135140 try {
136- response = autoTx ? request .executeWithinTransaction (connection .isImplicitTransactionSupported ())
137- : request .transaction (connection .getTransaction ()).executeAndWait ();
141+ response = sendRequest (stmt .getSQL (), r -> r );
138142 } catch (Exception e ) {
139143 throw SqlExceptionUtils .handle (e );
140144 } finally {
@@ -272,7 +276,6 @@ protected ClickHouseResponse processSqlStatement(ClickHouseSqlStatement stmt) th
272276
273277 protected ClickHouseResponse executeStatement (String stmt , Map <ClickHouseOption , Serializable > options ,
274278 List <ClickHouseExternalTable > tables , Map <String , String > settings ) throws SQLException {
275- boolean autoTx = connection .getAutoCommit () && connection .isTransactionSupported ();
276279 try {
277280 if (options != null ) {
278281 request .options (options );
@@ -310,9 +313,8 @@ protected ClickHouseResponse executeStatement(String stmt, Map<ClickHouseOption,
310313 }
311314 request .external (list );
312315 }
313- request .query (stmt , queryId = connection .newQueryId ());
314- return autoTx ? request .executeWithinTransaction (connection .isImplicitTransactionSupported ())
315- : request .transaction (connection .getTransaction ()).executeAndWait ();
316+
317+ return sendRequest (stmt , r -> r );
316318 } catch (Exception e ) {
317319 throw SqlExceptionUtils .handle (e );
318320 }
@@ -328,18 +330,61 @@ protected ClickHouseResponse executeStatement(ClickHouseSqlStatement stmt,
328330 return executeStatement (stmt .getSQL (), options , tables , settings );
329331 }
330332
331- protected int executeInsert (String sql , InputStream input ) throws SQLException {
333+ private ClickHouseResponse sendRequest (String sql , Function < ClickHouseRequest <?>, ClickHouseRequest <?>> preSeal ) throws SQLException {
332334 boolean autoTx = connection .getAutoCommit () && connection .isTransactionSupported ();
333- Mutation req = request .write ().query (sql , queryId = connection .newQueryId ()).data (input );
334- try (ClickHouseResponse resp = autoTx
335- ? req .executeWithinTransaction (connection .isImplicitTransactionSupported ())
336- : req .transaction (connection .getTransaction ()).executeAndWait ();
337- ResultSet rs = updateResult (new ClickHouseSqlStatement (sql , StatementType .INSERT ), resp )) {
338- // ignore
335+
336+ ClickHouseRequest <?> req ;
337+ ClickHouseTransaction tx = null ;
338+ synchronized (request ) {
339+ try {
340+ if (autoTx ) {
341+ if (connection .isImplicitTransactionSupported ()) {
342+ request .set (ClickHouseTransaction .SETTING_IMPLICIT_TRANSACTION , 1 ).transaction (null );
343+ } else {
344+ tx = request .getManager ().createImplicitTransaction (request );
345+ request .transaction (connection .getTransaction ());
346+ }
347+ } else {
348+ try {
349+ request .transaction (connection .getTransaction ());
350+ } catch (ClickHouseException e ) {
351+ throw SqlExceptionUtils .handle (e );
352+ }
353+ }
354+
355+ req = preSeal .apply (request ).query (sql , queryId = connection .newQueryId ()).seal ();
356+ } catch (Exception e ) {
357+ throw SqlExceptionUtils .handle (e );
358+ }
359+ }
360+
361+ try {
362+ return req .executeAndWait ();
339363 } catch (Exception e ) {
364+ if (tx != null ) {
365+ try {
366+ tx .rollback ();
367+ } catch (Exception ex ) {
368+ log .warn ("Failed to rollback transaction" , ex );
369+ }
370+ }
340371 throw SqlExceptionUtils .handle (e );
372+ } finally {
373+ try {
374+ request .transaction (null );
375+ } catch (Exception e ) {
376+ throw SqlExceptionUtils .handle (ClickHouseException .of (e , req .getServer ()));
377+ }
341378 }
379+ }
342380
381+ protected int executeInsert (String sql , InputStream input ) throws SQLException {
382+ try (ClickHouseResponse response = sendRequest (sql , r -> r .write ().data (input ));
383+ ResultSet rs = updateResult (new ClickHouseSqlStatement (sql , StatementType .INSERT ), response )) {
384+ // no more actions needed
385+ } catch (Exception e ) {
386+ throw SqlExceptionUtils .handle (e );
387+ }
343388 return (int ) currentUpdateCount ;
344389 }
345390
0 commit comments