Skip to content

Commit 562db61

Browse files
committed
implemented simple batch for prepared statement
1 parent 710f51f commit 562db61

File tree

4 files changed

+213
-72
lines changed

4 files changed

+213
-72
lines changed

jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java

Lines changed: 72 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import java.sql.SQLFeatureNotSupportedException;
3232
import java.sql.SQLType;
3333
import java.sql.SQLXML;
34+
import java.sql.Statement;
3435
import java.sql.Time;
3536
import java.sql.Timestamp;
3637
import java.sql.Types;
@@ -52,6 +53,7 @@
5253
import java.util.List;
5354
import java.util.Map;
5455
import java.util.UUID;
56+
import java.util.stream.IntStream;
5557

5658
public class PreparedStatementImpl extends StatementImpl implements PreparedStatement, JdbcV2Wrapper {
5759
private static final Logger LOG = LoggerFactory.getLogger(PreparedStatementImpl.class);
@@ -65,35 +67,60 @@ public class PreparedStatementImpl extends StatementImpl implements PreparedStat
6567
private final Calendar defaultCalendar;
6668

6769
private final String originalSql;
68-
private final String [] values;
70+
private final String[] values; // temp value holder (set can be called > once)
71+
private final List<StringBuilder> batchValues; // composed value statements
6972
private final ParsedPreparedStatement parsedPreparedStatement;
73+
private final boolean insertStmtWithValues;
74+
private final String valueListTmpl;
75+
private final int[] paramPositionsInDataClause;
76+
77+
private final int argCount;
7078

7179
private final ParameterMetaData parameterMetaData;
7280
private ResultSetMetaData resultSetMetaData = null;
7381

7482
public PreparedStatementImpl(ConnectionImpl connection, String sql, ParsedPreparedStatement parsedStatement) throws SQLException {
7583
super(connection);
7684
this.isPoolable = true; // PreparedStatement is poolable by default
77-
this.originalSql = sql.trim();
85+
this.originalSql = sql;
7886
this.parsedPreparedStatement = parsedStatement;
87+
this.argCount = parsedStatement.getArgCount();
7988

8089
this.defaultCalendar = connection.defaultCalendar;
81-
this.values = new String[parsedStatement.getArgCount()];
90+
this.values = new String[argCount];
8291
this.parameterMetaData = new ParameterMetaDataImpl(this.values.length);
92+
93+
int valueListStartPos = parsedStatement.getAssignValuesListStartPosition();
94+
int valueListStopPos = parsedStatement.getAssignValuesListStopPosition();
95+
if (valueListStartPos > -1 && valueListStopPos > -1) {
96+
int[] positions = parsedStatement.getParamPositions();
97+
paramPositionsInDataClause = new int[argCount];
98+
for (int i = 0; i < argCount; i++) {
99+
int p = positions[i] - valueListStartPos;
100+
paramPositionsInDataClause[i] = p;
101+
}
102+
103+
valueListTmpl = originalSql.substring(valueListStartPos, valueListStopPos + 1);
104+
insertStmtWithValues = true;
105+
batchValues = new ArrayList<>();
106+
} else {
107+
paramPositionsInDataClause = new int[0];
108+
batchValues = Collections.emptyList();
109+
valueListTmpl = "";
110+
insertStmtWithValues = false;
111+
}
83112
}
84113

85114
private String buildSQL() {
86115
StringBuilder compiledSql = new StringBuilder(originalSql);
87116
int posOffset = 0;
88117
int[] positions = parsedPreparedStatement.getParamPositions();
89-
for (int i = 0; i < parsedPreparedStatement.getArgCount(); i++) {
118+
for (int i = 0; i < argCount; i++) {
90119
int p = positions[i] + posOffset;
91120
String val = values[i].toString();
92121
compiledSql.replace(p, p+1, val);
93122
posOffset += val.length() - 1;
94123
}
95-
96-
97124
return compiledSql.toString();
98125
}
99126

@@ -246,28 +273,58 @@ public boolean execute() throws SQLException {
246273
public void addBatch() throws SQLException {
247274
checkClosed();
248275

276+
if (insertStmtWithValues) {
277+
StringBuilder valuesClause = new StringBuilder(valueListTmpl);
278+
int posOffset = 0;
279+
for (int i = 0; i < argCount; i++) {
280+
int p = paramPositionsInDataClause[i] + posOffset;
281+
valuesClause.replace(p, p + 1, values[i]);
282+
posOffset += values[i].length() - 1;
283+
}
284+
batchValues.add(valuesClause);
285+
} else {
286+
addBatch(buildSQL());
287+
}
249288
}
250289

251290
@Override
252291
public int[] executeBatch() throws SQLException {
253292
checkClosed();
254293

255-
// run executeBatch
256-
return executeBatchImpl().stream().mapToInt(Integer::intValue).toArray();
257-
294+
if (insertStmtWithValues) {
295+
// run executeBatch
296+
return executeInsertBatch().stream().mapToInt(Integer::intValue).toArray();
297+
} else {
298+
return super.executeBatch();
299+
}
258300
}
259301

260302
@Override
261303
public long[] executeLargeBatch() throws SQLException {
262-
return executeBatchImpl().stream().mapToLong(Integer::longValue).toArray();
304+
checkClosed();
305+
306+
if (insertStmtWithValues) {
307+
return executeInsertBatch().stream().mapToLong(Integer::longValue).toArray();
308+
} else {
309+
return super.executeLargeBatch();
310+
}
263311
}
264312

265-
private List<Integer> executeBatchImpl() throws SQLException {
266-
List<Integer> results = new ArrayList<>();
267-
for (String sql : batch) {
268-
results.add(executeUpdateImpl(sql, localSettings));
313+
private List<Integer> executeInsertBatch() throws SQLException {
314+
StringBuilder insertSql = new StringBuilder(originalSql.substring(0,
315+
parsedPreparedStatement.getAssignValuesListStartPosition()));
316+
317+
for (StringBuilder valuesList : batchValues) {
318+
insertSql.append(valuesList).append(',');
319+
}
320+
insertSql.setLength(insertSql.length() - 1);
321+
322+
int updateCount = super.executeUpdateImpl(insertSql.toString(), localSettings);
323+
if (updateCount == batchValues.size()) {
324+
return Collections.nCopies(batchValues.size(), 1);
325+
} else {
326+
return Collections.nCopies(batchValues.size(), Statement.SUCCESS_NO_INFO);
269327
}
270-
return results;
271328
}
272329

273330
@Override

jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/ParsedPreparedStatement.java

Lines changed: 17 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,9 @@ public class ParsedPreparedStatement extends ClickHouseParserBaseListener {
3232

3333
private int[] paramPositions = new int[16];
3434

35-
private int insertValuesClausePos = -1;
35+
private int assignValuesListStartPosition = -1;
36+
37+
private int assignValuesListStopPosition = -1;
3638

3739
public void setHasResultSet(boolean hasResultSet) {
3840
this.hasResultSet = hasResultSet;
@@ -70,10 +72,6 @@ public String getTable() {
7072
return table;
7173
}
7274

73-
public int getInsertValuesClausePos() {
74-
return insertValuesClausePos;
75-
}
76-
7775
public int[] getParamPositions() {
7876
return paramPositions;
7977
}
@@ -86,6 +84,14 @@ public boolean isCanStream() {
8684
return canStream;
8785
}
8886

87+
public int getAssignValuesListStartPosition() {
88+
return assignValuesListStartPosition;
89+
}
90+
91+
public int getAssignValuesListStopPosition() {
92+
return assignValuesListStopPosition;
93+
}
94+
8995
@Override
9096
public void enterColumnExprParam(ClickHouseParser.ColumnExprParamContext ctx) {
9197
appendParameter(ctx.start.getStartIndex());
@@ -101,6 +107,12 @@ public void enterInsertParameterFuncExpr(ClickHouseParser.InsertParameterFuncExp
101107
hasFuncWrappedParameter = true;
102108
}
103109

110+
@Override
111+
public void enterAssignmentValuesList(ClickHouseParser.AssignmentValuesListContext ctx) {
112+
assignValuesListStartPosition = ctx.getStart().getStartIndex();
113+
assignValuesListStopPosition = ctx.getStop().getStopIndex();
114+
}
115+
104116
@Override
105117
public void enterInsertParameter(ClickHouseParser.InsertParameterContext ctx) {
106118
appendParameter(ctx.start.getStartIndex());
@@ -117,11 +129,6 @@ private void appendParameter(int startIndex) {
117129
}
118130
}
119131

120-
@Override
121-
public void exitDataClauseValues(ClickHouseParser.DataClauseValuesContext ctx) {
122-
insertValuesClausePos = ctx.VALUES().getSymbol().getStopIndex();
123-
}
124-
125132
@Override
126133
public void enterInsertStmt(ClickHouseParser.InsertStmtContext ctx) {
127134
ClickHouseParser.TableIdentifierContext tableId = ctx.tableIdentifier();

jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java

Lines changed: 95 additions & 41 deletions
Original file line numberDiff line numberDiff line change
@@ -606,8 +606,17 @@ void testClearParameters() throws Exception {
606606
}
607607
}
608608

609-
@Test
610-
void testBatchInsert() throws Exception {
609+
@DataProvider
610+
Object[][] testBatchInsertWithRowBinary_dp() {
611+
return new Object[][]{
612+
{"INSERT INTO \n `%s` \nVALUES (?, ?, abs(?), ?)", PreparedStatementImpl.class}, // only string possible
613+
{"INSERT INTO\n `%s` \nVALUES (?, ?, ?, ?)", WriterStatementImpl.class}, // row binary writer
614+
{" INSERT INTO %s (ts, v1, v2, v3) VALUES (?, ?, ?, ?)", PreparedStatementImpl.class} // only string supported now
615+
};
616+
}
617+
618+
@Test(dataProvider = "testBatchInsertWithRowBinary_dp")
619+
void testBatchInsertWithRowBinary(String sql, Class implClass) throws Exception {
611620
String table = "test_batch";
612621
long seed = System.currentTimeMillis();
613622
Random rnd = new Random(seed);
@@ -621,52 +630,97 @@ void testBatchInsert() throws Exception {
621630
" ( ts DateTime, v1 Int32, v2 Float32, v3 Int32) Engine MergeTree ORDER BY ()");
622631
}
623632

624-
String[] sql = new String[]{
625-
"INSERT INTO \n `%s` \nVALUES (?, ?, multiply(?, 10), ?)", // only string possible
626-
"INSERT INTO\n `%s` \nVALUES (?, ?, ?, ?)", // row binary writer
627-
" INSERT INTO %s (ts, v1, v2, v3) VALUES (?, ?, ?, ?)", // only string supported now
628-
};
629-
Class<?>[] impl = new Class<?>[]{
630-
PreparedStatementImpl.class,
631-
WriterStatementImpl.class,
632-
PreparedStatementImpl.class
633-
};
634-
635-
for (int i = 0; i < sql.length; i++) {
636-
final int nBatches = 10;
637-
try (PreparedStatement stmt = conn.prepareStatement(String.format(sql[i], table))) {
638-
Assert.assertEquals(stmt.getClass(), impl[i]);
639-
for (int bI = 0; bI < nBatches; bI++) {
640-
stmt.setTimestamp(1, Timestamp.valueOf(LocalDateTime.now()));
641-
stmt.setInt(2, rnd.nextInt());
642-
stmt.setFloat(3, rnd.nextFloat());
643-
stmt.setInt(4, rnd.nextInt());
644-
stmt.addBatch();
645-
}
633+
final int nBatches = 10;
634+
try (PreparedStatement stmt = conn.prepareStatement(String.format(sql, table))) {
635+
Assert.assertEquals(stmt.getClass(), implClass);
636+
for (int bI = 0; bI < nBatches; bI++) {
637+
stmt.setTimestamp(1, Timestamp.valueOf(LocalDateTime.now()));
638+
stmt.setInt(2, rnd.nextInt());
639+
stmt.setFloat(3, rnd.nextFloat());
640+
stmt.setInt(4, rnd.nextInt());
641+
stmt.addBatch();
642+
}
646643

647-
int[] result = stmt.executeBatch();
648-
for (int r : result) {
649-
Assert.assertEquals(r, 1);
650-
}
644+
int[] result = stmt.executeBatch();
645+
for (int r : result) {
646+
Assert.assertEquals(r, 1);
651647
}
648+
}
652649

653-
try (Statement stmt = conn.createStatement();
654-
ResultSet rs = stmt.executeQuery("SELECT * FROM " + table);) {
650+
try (Statement stmt = conn.createStatement();
651+
ResultSet rs = stmt.executeQuery("SELECT * FROM " + table);) {
655652

656-
int count = 0;
657-
while (rs.next()) {
658-
Timestamp ts = rs.getTimestamp(1);
659-
assertNotNull(ts);
660-
assertTrue(rs.getInt(2) != 0);
661-
assertTrue(rs.getFloat(3) != 0.0f);
662-
assertTrue(rs.getInt(4) != 0);
663-
count++;
664-
}
665-
assertEquals(count, nBatches);
653+
int count = 0;
654+
while (rs.next()) {
655+
Timestamp ts = rs.getTimestamp(1);
656+
assertNotNull(ts);
657+
assertTrue(rs.getInt(2) != 0);
658+
assertTrue(rs.getFloat(3) != 0.0f);
659+
assertTrue(rs.getInt(4) != 0);
660+
count++;
661+
}
662+
assertEquals(count, nBatches);
663+
664+
stmt.execute("TRUNCATE " + table);
665+
}
666+
}
667+
}
668+
669+
@DataProvider
670+
Object[][] testBatchInsertTextStatement_dp() {
671+
return new Object[][]{
672+
{"INSERT INTO \n `%s` \nVALUES (?, ?, ?, ?)"}, // simple
673+
{" INSERT INTO %s (ts, v1, v2, v3) VALUES (?, ?, ?, ?)"},
674+
};
675+
}
676+
677+
@Test(dataProvider = "testBatchInsertTextStatement_dp")
678+
void testBatchInsertTextStatement(String sql) throws Exception {
679+
String table = "test_batch_text";
680+
long seed = System.currentTimeMillis();
681+
Random rnd = new Random(seed);
682+
System.out.println("testBatchInsert seed" + seed);
683+
try (Connection conn = getJdbcConnection()) {
684+
685+
try (Statement stmt = conn.createStatement()) {
686+
stmt.execute("CREATE TABLE IF NOT EXISTS " + table +
687+
" ( ts DateTime DEFAULT now(), v1 Int32, v2 Float32, v3 Int32) Engine MergeTree ORDER BY ()");
688+
}
666689

667-
stmt.execute("TRUNCATE " + table);
690+
final int nBatches = 10;
691+
try (PreparedStatement stmt = conn.prepareStatement(String.format(sql, table))) {
692+
Assert.assertEquals(stmt.getClass(), PreparedStatementImpl.class);
693+
for (int bI = 0; bI < nBatches; bI++) {
694+
stmt.setTimestamp(1, Timestamp.valueOf(LocalDateTime.now()));
695+
stmt.setInt(2, rnd.nextInt());
696+
stmt.setFloat(3, rnd.nextFloat());
697+
stmt.setInt(4, rnd.nextInt());
698+
stmt.addBatch();
699+
}
700+
701+
int[] result = stmt.executeBatch();
702+
for (int r : result) {
703+
Assert.assertEquals(r, 1);
668704
}
669705
}
706+
707+
try (Statement stmt = conn.createStatement();
708+
ResultSet rs = stmt.executeQuery("SELECT * FROM " + table);) {
709+
710+
int count = 0;
711+
while (rs.next()) {
712+
Timestamp ts = rs.getTimestamp(1);
713+
assertNotNull(ts);
714+
assertTrue(rs.getInt(2) != 0);
715+
assertTrue(rs.getFloat(3) != 0.0f);
716+
assertTrue(rs.getInt(4) != 0);
717+
count++;
718+
}
719+
assertEquals(count, nBatches);
720+
721+
stmt.execute("TRUNCATE " + table);
722+
}
723+
670724
}
671725
}
672726

0 commit comments

Comments
 (0)