-
Notifications
You must be signed in to change notification settings - Fork 624
Expand file tree
/
Copy pathStatementImpl.java
More file actions
681 lines (586 loc) · 24.2 KB
/
StatementImpl.java
File metadata and controls
681 lines (586 loc) · 24.2 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
package com.clickhouse.jdbc;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
import com.clickhouse.client.api.internal.ServerSettings;
import com.clickhouse.client.api.query.QueryResponse;
import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.client.api.sql.SQLUtils;
import com.clickhouse.jdbc.internal.ExceptionUtils;
import com.clickhouse.jdbc.internal.FeatureManager;
import com.clickhouse.jdbc.internal.ParsedStatement;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.SocketTimeoutException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.SQLWarning;
import java.sql.Statement;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
public class StatementImpl implements Statement, JdbcV2Wrapper {
private static final Logger LOG = LoggerFactory.getLogger(StatementImpl.class);
// Attributes
ConnectionImpl connection;
protected int queryTimeout;
protected boolean isPoolable = false; // Statement is not poolable by default
private final FeatureManager featureManager;
// State
private volatile boolean closed;
private final ConcurrentLinkedQueue<ResultSetImpl> resultSets; // all result sets linked to this statement
protected ResultSetImpl currentResultSet;
protected long currentUpdateCount = -1;
protected List<String> batch;
private String lastStatementSql;
private ParsedStatement parsedStatement;
protected volatile String lastQueryId;
private long maxRows;
private boolean closeOnCompletion;
private final boolean resultSetAutoClose;
private int maxFieldSize;
private boolean escapeProcessingEnabled;
private final Supplier<String> queryIdGenerator;
private int fetchSize = 1;
// settings local to a statement
protected QuerySettings localSettings;
public StatementImpl(ConnectionImpl connection) throws SQLException {
this.connection = connection;
this.queryTimeout = 0;
this.closed = false;
this.batch = new ArrayList<>();
this.maxRows = 0;
this.localSettings = QuerySettings.merge(connection.getDefaultQuerySettings(), new QuerySettings());
this.resultSets= new ConcurrentLinkedQueue<>();
this.resultSetAutoClose = connection.getJdbcConfig().isSet(DriverProperties.RESULTSET_AUTO_CLOSE);
this.escapeProcessingEnabled = true;
this.featureManager = new FeatureManager(connection.getJdbcConfig());
this.queryIdGenerator = connection.getJdbcConfig().getQueryIdGenerator();
}
protected void ensureOpen() throws SQLException {
if (closed) {
throw new SQLException("Statement is closed", ExceptionUtils.SQL_STATE_CONNECTION_EXCEPTION);
}
}
private String parseJdbcEscapeSyntax(String sql) {
LOG.trace("Original SQL: {}", sql);
if (escapeProcessingEnabled) {
sql = escapedSQLToNative(sql);
}
LOG.trace("Escaped SQL: {}", sql);
return sql;
}
public static String escapedSQLToNative(String sql) {
if (sql == null) {
throw new IllegalArgumentException("SQL may not be null");
}
// Replace {d 'YYYY-MM-DD'} with corresponding SQL date format
sql = sql.replaceAll("\\{d '([^']*)'\\}", "toDate('$1')");
// Replace {ts 'YYYY-MM-DD HH:mm:ss'} with corresponding SQL timestamp format
sql = sql.replaceAll("\\{ts '([^']*)'\\}", "timestamp('$1')");
// Replace function escape syntax {fn <function>} (e.g., {fn UCASE(name)})
sql = sql.replaceAll("\\{fn ([^\\}]*)\\}", "$1");
// Handle outer escape syntax
//sql = sql.replaceAll("\\{escape '([^']*)'\\}", "'$1'");
// Note: do not remove new lines because they may be used to delimit comments
// Add more replacements as needed for other JDBC escape sequences
return sql;
}
protected String getLastStatementSql() {
return lastStatementSql;
}
@Override
public ResultSet executeQuery(String sql) throws SQLException {
ensureOpen();
currentUpdateCount = -1;
currentResultSet = executeQueryImpl(sql, localSettings);
if (currentResultSet == null) {
throw new SQLException("executeQuery() requires a ResultSet; use execute() for statements that do not produce one.",
ExceptionUtils.SQL_STATE_CLIENT_ERROR);
}
return currentResultSet;
}
private void closeCurrentResultSet() {
if (currentResultSet != null) {
LOG.debug("Previous result set is open [resultSet = " + currentResultSet + "]");
// Closing request blindly assuming that user do not care about it anymore (DDL request for example)
try {
currentResultSet.close();
} catch (Exception e) {
LOG.error("Failed to close previous result set", e);
} finally {
currentResultSet = null; // no need to remember we have closed it already
}
}
}
/**
* Sets last queryId and returns actual query Id
* Accepts null
* @param queryId
* @return
*/
protected String setLastQueryID(String queryId) {
if (queryId == null) {
queryId = queryIdGenerator == null ? UUID.randomUUID().toString() : queryIdGenerator.get();
}
lastQueryId = queryId;
LOG.debug("Query ID: {}", lastQueryId);
return queryId;
}
protected ResultSetImpl executeQueryImpl(String sql, QuerySettings settings) throws SQLException {
ensureOpen();
// Closing before trying to do next request. Otherwise, deadlock because previous connection will not be
// release before this one completes.
if (resultSetAutoClose) {
closeCurrentResultSet();
// There is a feature `closeOnComplete` that dictates closing the statement when all
// result sets are closed. Call to `closeCurrentResultSet` will trigger this statement
// closure. But it should not happen because this was introduces instead of spec and will be removed in the future.
// So we need to make this statement open again because we're going to create a new result set.
this.closed = false;
}
QuerySettings mergedSettings = QuerySettings.merge(settings, new QuerySettings());
mergedSettings.setQueryId(setLastQueryID(mergedSettings.getQueryId()));
QueryResponse response = null;
try {
lastStatementSql = parseJdbcEscapeSyntax(sql);
LOG.trace("SQL Query: {}", lastStatementSql); // this is not secure for create statements because of passwords
if (queryTimeout == 0) {
response = connection.getClient().query(lastStatementSql, mergedSettings).get();
} else {
response = connection.getClient().query(lastStatementSql, mergedSettings).get(queryTimeout, TimeUnit.SECONDS);
}
if (response.getFormat().isText()) {
throw new SQLException("Only RowBinaryWithNameAndTypes is supported for output format. Please check your query.",
ExceptionUtils.SQL_STATE_CLIENT_ERROR);
}
ClickHouseBinaryFormatReader reader = connection.getClient().newBinaryFormatReader(response);
if (reader.getSchema() == null) {
long writtenRows = 0L;
try {
writtenRows = response.getWrittenRows();
} catch (Exception ignore) {
// Best effort: leave writtenRows as 0 if we can't obtain it.
}
this.currentUpdateCount = (int) Math.min(writtenRows, Integer.MAX_VALUE);
try {
reader.close();
} catch (Exception closeEx) {
LOG.warn("Failed to close reader when schema is null", closeEx);
} finally {
try {
response.close();
} catch (Exception closeRespEx) {
LOG.warn("Failed to close response when schema is null", closeRespEx);
}
}
onResultSetClosed(null);
return null;
}
return new ResultSetImpl(this, response, reader, this::handleSocketTimeoutException);
} catch (Exception e) {
if (response != null) {
try {
response.close();
} catch (Exception ex) {
LOG.warn("Failed to close response after exception", e);
}
}
handleSocketTimeoutException(e);
onResultSetClosed(null);
throw ExceptionUtils.toSqlState(e);
}
}
protected void handleSocketTimeoutException(Exception e) {
if (e.getCause() instanceof SocketTimeoutException || e instanceof SocketTimeoutException) {
this.connection.onNetworkTimeout();
}
}
@Override
public int executeUpdate(String sql) throws SQLException {
ensureOpen();
return (int)executeLargeUpdate(sql);
}
protected long executeUpdateImpl(String sql, QuerySettings settings) throws SQLException {
ensureOpen();
// Closing before trying to do next request. Otherwise, deadlock because previous connection will not be
// release before this one completes.
if (resultSetAutoClose) {
closeCurrentResultSet();
}
QuerySettings mergedSettings = QuerySettings.merge(connection.getDefaultQuerySettings(), settings);
mergedSettings.setQueryId(setLastQueryID(mergedSettings.getQueryId()));
lastStatementSql = parseJdbcEscapeSyntax(sql);
LOG.trace("SQL Query: {}", lastStatementSql);
int updateCount = 0;
try (QueryResponse response = queryTimeout == 0 ? connection.getClient().query(lastStatementSql, mergedSettings).get()
: connection.getClient().query(lastStatementSql, mergedSettings).get(queryTimeout, TimeUnit.SECONDS)) {
updateCount = Math.max(0, (int) response.getWrittenRows()); // when statement alters schema no result rows returned.
lastQueryId = response.getQueryId();
} catch (Exception e) {
handleSocketTimeoutException(e);
throw ExceptionUtils.toSqlState(e);
}
return updateCount;
}
private void postUpdateActions() throws SQLException {
if (parsedStatement.getUseDatabase() != null) {
this.localSettings.setDatabase(parsedStatement.getUseDatabase());
}
if (parsedStatement.getRoles() != null) {
this.connection.getClient().setDBRoles(parsedStatement.getRoles());
this.localSettings.setDBRoles(parsedStatement.getRoles());
}
}
@Override
public void close() throws SQLException {
closed = true;
closeCurrentResultSet();
for (ResultSetImpl resultSet : resultSets) {
if (resultSet != null && !resultSet.isClosed()) {
try {
resultSet.close();
} catch (Exception e) {
LOG.error("Failed to close result set", e);
}
}
}
}
@Override
public int getMaxFieldSize() throws SQLException {
ensureOpen();
return this.maxFieldSize;
}
@Override
public void setMaxFieldSize(int max) throws SQLException {
ensureOpen();
if (max < 0) {
throw new SQLException("max should be a positive integer.");
}
this.maxFieldSize = max;
}
@Override
public int getMaxRows() throws SQLException {
ensureOpen();
return (int) getLargeMaxRows(); // skip overflow check.
}
@Override
public void setMaxRows(int max) throws SQLException {
setLargeMaxRows(max);
}
@Override
public void setEscapeProcessing(boolean enable) throws SQLException {
ensureOpen();
this.escapeProcessingEnabled = enable;
}
@Override
public int getQueryTimeout() throws SQLException {
ensureOpen();
return queryTimeout;
}
@Override
public void setQueryTimeout(int seconds) throws SQLException {
ensureOpen();
queryTimeout = seconds;
}
@Override
public void cancel() throws SQLException {
if (closed) {
return;
}
// KILL QUERY must not run inside the same session as the query being canceled otherwise it will
// cause "Session is locked by a concurrent client" (SESSION_IS_LOCKED) error.
QuerySettings cancelSettings = QuerySettings.merge(getLocalSettings(), new QuerySettings()).clearSession();
try (QueryResponse response = connection.getClient().query(String.format("KILL QUERY%sWHERE query_id = '%s'",
connection.onCluster ? " ON CLUSTER " + SQLUtils.enquoteIdentifier(connection.cluster, true) + ' ' : ' ',
lastQueryId), cancelSettings).get()){
LOG.debug("Query {} was killed by {}", lastQueryId, response.getQueryId());
} catch (Exception e) {
throw new SQLException(e);
}
}
@Override
public SQLWarning getWarnings() throws SQLException {
ensureOpen();
return null;
}
@Override
public void clearWarnings() throws SQLException {
ensureOpen();
}
@Override
public void setCursorName(String name) throws SQLException {
featureManager.unsupportedFeatureThrow("setCursorName(String)", true);
ensureOpen();
}
@Override
public boolean execute(String sql) throws SQLException {
ensureOpen();
parsedStatement = connection.getSqlParser().parsedStatement(sql);
currentUpdateCount = -1;
if (parsedStatement.isHasResultSet()) {
currentResultSet = executeQueryImpl(sql, localSettings);
return currentResultSet != null;
} else {
currentUpdateCount = executeUpdateImpl(sql, localSettings);
postUpdateActions();
return false;
}
}
@Override
public ResultSet getResultSet() throws SQLException {
ensureOpen();
return currentResultSet;
}
@Override
public int getUpdateCount() throws SQLException {
ensureOpen();
return (int) getLargeUpdateCount();
}
@Override
public boolean getMoreResults() throws SQLException {
ensureOpen();
return getMoreResults(Statement.CLOSE_CURRENT_RESULT);
}
@Override
public void setFetchDirection(int direction) throws SQLException {
ensureOpen();
if (direction != ResultSet.FETCH_FORWARD && direction != ResultSet.FETCH_REVERSE && direction != ResultSet.FETCH_UNKNOWN) {
throw new SQLException("Invalid fetch direction: " + direction + ". Should be one of ResultSet.FETCH_FORWARD, ResultSet.FETCH_REVERSE, or ResultSet.FETCH_UNKNOWN");
}
}
@Override
public int getFetchDirection() throws SQLException {
ensureOpen();
return ResultSet.FETCH_FORWARD;
}
@Override
public void setFetchSize(int rows) throws SQLException {
ensureOpen();
if (rows < 0) {
throw new SQLException("rows should be greater or equal to 0.");
}
this.fetchSize = rows;
}
@Override
public int getFetchSize() throws SQLException {
ensureOpen();
return fetchSize;
}
@Override
public int getResultSetConcurrency() throws SQLException {
ensureOpen();
return ResultSet.CONCUR_READ_ONLY;
}
@Override
public int getResultSetType() throws SQLException {
ensureOpen();
return ResultSet.TYPE_FORWARD_ONLY;
}
@Override
public void addBatch(String sql) throws SQLException {
ensureOpen();
batch.add(sql);
}
@Override
public void clearBatch() throws SQLException {
ensureOpen();
batch.clear();
}
@Override
public int[] executeBatch() throws SQLException {
return executeBatchImpl().stream().mapToInt(i -> i).toArray();
}
private List<Integer> executeBatchImpl() throws SQLException {
ensureOpen();
List<Integer> results = new ArrayList<>();
for (String sql : batch) {
results.add(executeUpdate(sql));
}
clearBatch();
return results;
}
@Override
public ConnectionImpl getConnection() throws SQLException {
return connection;
}
/**
* Returns instance of local settings. Can be used to override settings.
*
* @return QuerySettings that is used as base for each request.
*/
public QuerySettings getLocalSettings() {
return localSettings;
}
@Override
public boolean getMoreResults(int current) throws SQLException {
// This method designed to iterate over multiple resultsets after "execute(sql)" method is called
// But we have at most only one always
// Then we should close any existing and return false to indicate that no more result are present
if (currentResultSet != null && current != Statement.KEEP_CURRENT_RESULT) {
currentResultSet.close();
}
currentResultSet = null;
currentUpdateCount = -1;
return false; // false indicates that no more results (or it is an update count)
}
// @Override -- because doesn't exist in Java 8
public String enquoteLiteral(String val) throws SQLException {
return SQLUtils.enquoteLiteral(val);
}
// @Override -- because doesn't exist in Java 8
public String enquoteIdentifier(String identifier, boolean alwaysQuote) throws SQLException {
return SQLUtils.enquoteIdentifier(identifier, alwaysQuote);
}
// @Override -- because doesn't exist in Java 8
public boolean isSimpleIdentifier(String identifier) throws SQLException {
return SQLUtils.isSimpleIdentifier(identifier);
}
// @Override -- because doesn't exist in Java 8
public String enquoteNCharLiteral(String val) throws SQLException {
if (val == null) {
throw new NullPointerException();
}
return "N" + SQLUtils.enquoteLiteral(val);
}
@Override
public ResultSet getGeneratedKeys() throws SQLException {
// TODO: return empty result set or throw exception
return null;
}
@Override
public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
featureManager.unsupportedFeatureThrow("executeUpdate(String, int)", autoGeneratedKeys != Statement.NO_GENERATED_KEYS);
return executeUpdate(sql);
}
@Override
public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
featureManager.unsupportedFeatureThrow("executeUpdate(String, int[])");
return executeUpdate(sql);
}
@Override
public int executeUpdate(String sql, String[] columnNames) throws SQLException {
featureManager.unsupportedFeatureThrow("executeUpdate(String, String[])");
return executeUpdate(sql);
}
@Override
public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
featureManager.unsupportedFeatureThrow("execute(String, int)", autoGeneratedKeys != Statement.NO_GENERATED_KEYS);
return execute(sql);
}
@Override
public boolean execute(String sql, int[] columnIndexes) throws SQLException {
featureManager.unsupportedFeatureThrow("execute(String, int[])");
return execute(sql);
}
@Override
public boolean execute(String sql, String[] columnNames) throws SQLException {
featureManager.unsupportedFeatureThrow("execute(String, String[])");
return execute(sql);
}
@Override
public int getResultSetHoldability() throws SQLException {
return ResultSet.HOLD_CURSORS_OVER_COMMIT; // we do not have transactions and result must be closed by app.
}
@Override
public boolean isClosed() throws SQLException {
return closed;
}
@Override
public void setPoolable(boolean poolable) throws SQLException {
ensureOpen();
this.isPoolable = poolable;
}
@Override
public boolean isPoolable() throws SQLException {
return isPoolable;
}
@Override
public void closeOnCompletion() throws SQLException {
ensureOpen();
this.closeOnCompletion = true;
}
// called each time query is complete or result set is closed
public void onResultSetClosed(ResultSetImpl resultSet) throws SQLException {
if (resultSet != null) {
this.resultSets.remove(resultSet);
}
if (this.closeOnCompletion) {
if ((resultSets.isEmpty()) && (currentResultSet == null || currentResultSet.isClosed())) {
// last result set is closed.
this.closed = true;
}
}
}
@Override
public boolean isCloseOnCompletion() throws SQLException {
return this.closeOnCompletion;
}
@Override
public long getLargeUpdateCount() throws SQLException {
ensureOpen();
return currentUpdateCount;
}
@Override
public void setLargeMaxRows(long max) throws SQLException {
ensureOpen();
maxRows = max;
if (connection.getJdbcConfig().isFlagSet(DriverProperties.USE_MAX_RESULT_ROWS)) {
// This method override user set overflow mode on purpose:
// 1. Spec clearly states that after calling this method with a limit > 0 all rows over limit are dropped.
// 2. Calling this method should not cause throwing exception for future queries what only `break` can guarantee
// 3. If user wants different behavior then they are can use connection properties.
if (max > 0) {
localSettings.setOption(ClientConfigProperties.serverSetting(ServerSettings.MAX_RESULT_ROWS), maxRows);
localSettings.setOption(ClientConfigProperties.serverSetting(ServerSettings.RESULT_OVERFLOW_MODE),
ServerSettings.RESULT_OVERFLOW_MODE_BREAK);
} else {
// overriding potential client settings (set thru connection setup)
// there is no no limit value so we use very large limit.
localSettings.setOption(ClientConfigProperties.serverSetting(ServerSettings.MAX_RESULT_ROWS), Long.MAX_VALUE);
localSettings.setOption(ClientConfigProperties.serverSetting(ServerSettings.RESULT_OVERFLOW_MODE),
ServerSettings.RESULT_OVERFLOW_MODE_BREAK);
}
}
}
@Override
public long getLargeMaxRows() throws SQLException {
ensureOpen();
return this.maxRows;
}
@Override
public long[] executeLargeBatch() throws SQLException {
return executeBatchImpl().stream().mapToLong(Integer::longValue).toArray();
}
@Override
public long executeLargeUpdate(String sql) throws SQLException {
parsedStatement = connection.getSqlParser().parsedStatement(sql);
long updateCount = executeUpdateImpl(sql, localSettings);
postUpdateActions();
return updateCount;
}
@Override
public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
featureManager.unsupportedFeatureThrow("executeLargeUpdate(String, int)", autoGeneratedKeys != Statement.NO_GENERATED_KEYS);
return executeLargeUpdate(sql);
}
@Override
public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException {
featureManager.unsupportedFeatureThrow("executeLargeUpdate(String, int[])");
return executeLargeUpdate(sql);
}
@Override
public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException {
featureManager.unsupportedFeatureThrow("executeLargeUpdate(String, String[])");
return executeLargeUpdate(sql);
}
/**
* Return query ID of last executed statement. It is not guaranteed when statements is used concurrently.
* @return query ID
*/
public String getLastQueryId() {
return lastQueryId;
}
}