Skip to content

Commit 49c8be1

Browse files
committed
passing query parameters in queryAll and queryRecords
1 parent 05a9985 commit 49c8be1

File tree

6 files changed

+298
-213
lines changed

6 files changed

+298
-213
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 48 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1731,36 +1731,53 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
17311731

17321732
return runAsyncOperation(responseSupplier, settings.getAllSettings());
17331733
}
1734+
public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Object> queryParams) {
1735+
return query(sqlQuery, queryParams, null);
1736+
}
17341737

17351738
/**
17361739
* <p>Queries data in one of descriptive format and creates a reader out of the response stream.</p>
17371740
* <p>Format is selected internally so is ignored when passed in settings. If query contains format
17381741
* statement then it may cause incompatibility error.</p>
17391742
*
1740-
* @param sqlQuery
1741-
* @return
1743+
* @param sqlQuery - SQL statement
1744+
* @return - a promise to a result
17421745
*/
17431746
public CompletableFuture<Records> queryRecords(String sqlQuery) {
1744-
return queryRecords(sqlQuery, null);
1747+
return queryRecords(sqlQuery, null, null);
17451748
}
17461749

17471750
/**
17481751
* <p>Queries data in one of descriptive format and creates a reader out of the response stream.</p>
17491752
* <p>Format is selected internally so is ignored when passed in settings. If query contains format
17501753
* statement then it may cause incompatibility error.</p>
17511754
*
1752-
* @param sqlQuery
1753-
* @param settings
1754-
* @return
1755+
* @param sqlQuery - SQL statement
1756+
* @param settings - operation settings
1757+
* @return - a promise to a result
17551758
*/
17561759
public CompletableFuture<Records> queryRecords(String sqlQuery, QuerySettings settings) {
1760+
return queryRecords(sqlQuery, null, settings);
1761+
}
1762+
1763+
/**
1764+
* <p>Queries data in one of descriptive format and creates a reader out of the response stream.</p>
1765+
* <p>Format is selected internally so is ignored when passed in settings. If query contains format
1766+
* statement then it may cause incompatibility error.</p>
1767+
* See {@link #query(String, Map, QuerySettings)} for parametrized queries.
1768+
* @param sqlQuery - SQL statement
1769+
* @param params - sql parameters
1770+
* @param settings - operation settings
1771+
* @return - a promise to a result
1772+
*/
1773+
public CompletableFuture<Records> queryRecords(String sqlQuery, Map<String, Object> params, QuerySettings settings) {
17571774
if (settings == null) {
17581775
settings = new QuerySettings();
17591776
}
17601777
settings.setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes);
17611778
settings.waitEndOfQuery(true); // we rely on the summery
17621779

1763-
return query(sqlQuery, settings).thenApply(response -> {
1780+
return query(sqlQuery, params, settings).thenApply(response -> {
17641781
try {
17651782

17661783
return new Records(response, newBinaryFormatReader(response));
@@ -1770,19 +1787,30 @@ public CompletableFuture<Records> queryRecords(String sqlQuery, QuerySettings se
17701787
});
17711788
}
17721789

1790+
public CompletableFuture<Records> queryRecords(String sqlQuery, Map<String, Object> params) {
1791+
return queryRecords(sqlQuery, params, null);
1792+
}
1793+
17731794
/**
17741795
* <p>Queries data in descriptive format and reads result to a collection.</p>
17751796
* <p>Use this method for queries that would return only a few records only because client
17761797
* will read whole dataset and convert it into a list of GenericRecord</p>
1777-
* @param sqlQuery - SQL query
1798+
*
1799+
* See {@link #query(String, Map, QuerySettings)} for parametrized queries.
1800+
* @param sqlQuery - SQL statement
1801+
* @param params - query parameters
1802+
* @param settings - operation settings
17781803
* @return - complete list of records
17791804
*/
1780-
public List<GenericRecord> queryAll(String sqlQuery, QuerySettings settings) {
1805+
public List<GenericRecord> queryAll(String sqlQuery, Map<String, Object> params, QuerySettings settings) {
1806+
if (settings == null) {
1807+
settings = new QuerySettings();
1808+
}
17811809
try {
17821810
int operationTimeout = getOperationTimeout();
17831811
settings.setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes)
17841812
.waitEndOfQuery(true);
1785-
try (QueryResponse response = operationTimeout == 0 ? query(sqlQuery, settings).get() :
1813+
try (QueryResponse response = operationTimeout == 0 ? query(sqlQuery, params, settings).get() :
17861814
query(sqlQuery, settings).get(operationTimeout, TimeUnit.MILLISECONDS)) {
17871815
List<GenericRecord> records = new ArrayList<>();
17881816
if (response.getResultRows() > 0) {
@@ -1791,7 +1819,7 @@ public List<GenericRecord> queryAll(String sqlQuery, QuerySettings settings) {
17911819

17921820
Map<String, Object> record;
17931821
while (reader.readRecord((record = new LinkedHashMap<>()))) {
1794-
records.add(new MapBackedRecord(record, reader.getSchema()));
1822+
records.add(new MapBackedRecord(record, reader.getConvertions(), reader.getSchema()));
17951823
}
17961824
}
17971825
return records;
@@ -1803,8 +1831,16 @@ public List<GenericRecord> queryAll(String sqlQuery, QuerySettings settings) {
18031831
}
18041832
}
18051833

1834+
public List<GenericRecord> queryAll(String sqlQuery, QuerySettings settings) {
1835+
return queryAll(sqlQuery, null, settings);
1836+
}
1837+
1838+
public List<GenericRecord> queryAll(String sqlQuery, Map<String, Object> params) {
1839+
return queryAll(sqlQuery, params, null);
1840+
}
1841+
18061842
public List<GenericRecord> queryAll(String sqlQuery) {
1807-
return queryAll(sqlQuery, new QuerySettings());
1843+
return queryAll(sqlQuery, null, (QuerySettings) null);
18081844
}
18091845

18101846
public <T> List<T> queryAll(String sqlQuery, Class<T> clazz, TableSchema schema) {

client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java

Lines changed: 17 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
package com.clickhouse.client.api.data_formats.internal;
22

3-
import com.clickhouse.client.api.ClientException;
43
import com.clickhouse.client.api.ClientConfigProperties;
4+
import com.clickhouse.client.api.ClientException;
55
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
66
import com.clickhouse.client.api.internal.MapUtils;
77
import com.clickhouse.client.api.internal.ServerSettings;
@@ -235,7 +235,6 @@ protected void setSchema(TableSchema schema) {
235235
for (int i = 0; i < columns.length; i++) {
236236
ClickHouseColumn column = columns[i];
237237

238-
Map<NumberType, Function<Object, ?>> converters = new HashMap<>();
239238
switch (column.getDataType()) {
240239
case Int8:
241240
case Int16:
@@ -257,21 +256,16 @@ protected void setSchema(TableSchema schema) {
257256
case Decimal128:
258257
case Decimal256:
259258
case Bool:
260-
converters.put(NumberType.Byte, SerializerUtils.NumberConverter::toByte);
261-
converters.put(NumberType.Short, SerializerUtils.NumberConverter::toShort);
262-
converters.put(NumberType.Int, SerializerUtils.NumberConverter::toInt);
263-
converters.put(NumberType.Long, SerializerUtils.NumberConverter::toLong);
264-
converters.put(NumberType.BigInteger, SerializerUtils.NumberConverter::toBigInteger);
265-
converters.put(NumberType.BigDecimal, SerializerUtils.NumberConverter::toBigDecimal);
266-
converters.put(NumberType.Float, SerializerUtils.NumberConverter::toFloat);
267-
converters.put(NumberType.Double, SerializerUtils.NumberConverter::toDouble);
268-
converters.put(NumberType.Boolean, SerializerUtils::convertToBoolean);
259+
this.convertions[i] = NumberConverter.NUMBER_CONVERTERS;
269260
break;
261+
default:
262+
this.convertions[i] = Collections.emptyMap();
270263
}
271-
272-
this.convertions[i] = converters;
273264
}
265+
}
274266

267+
public Map[] getConvertions() {
268+
return convertions;
275269
}
276270

277271
@Override
@@ -302,7 +296,7 @@ public String getString(int index) {
302296
return value.toString();
303297
}
304298

305-
private <T> T readNumberValue(String colName, NumberType targetType) {
299+
private <T> T readNumberValue(String colName, NumberConverter.NumberType targetType) {
306300
int colIndex = schema.nameToIndex(colName);
307301
Function<Object, Object> converter = (Function<Object, Object>) convertions[colIndex].get(targetType);
308302
if (converter != null) {
@@ -320,47 +314,47 @@ private <T> T readNumberValue(String colName, NumberType targetType) {
320314

321315
@Override
322316
public byte getByte(String colName) {
323-
return readNumberValue(colName, NumberType.Byte);
317+
return readNumberValue(colName, NumberConverter.NumberType.Byte);
324318
}
325319

326320
@Override
327321
public short getShort(String colName) {
328-
return readNumberValue(colName, NumberType.Short);
322+
return readNumberValue(colName, NumberConverter.NumberType.Short);
329323
}
330324

331325
@Override
332326
public int getInteger(String colName) {
333-
return readNumberValue(colName, NumberType.Int);
327+
return readNumberValue(colName, NumberConverter.NumberType.Int);
334328
}
335329

336330
@Override
337331
public long getLong(String colName) {
338-
return readNumberValue(colName, NumberType.Long);
332+
return readNumberValue(colName, NumberConverter.NumberType.Long);
339333
}
340334

341335
@Override
342336
public float getFloat(String colName) {
343-
return readNumberValue(colName, NumberType.Float);
337+
return readNumberValue(colName, NumberConverter.NumberType.Float);
344338
}
345339

346340
@Override
347341
public double getDouble(String colName) {
348-
return readNumberValue(colName, NumberType.Double);
342+
return readNumberValue(colName, NumberConverter.NumberType.Double);
349343
}
350344

351345
@Override
352346
public boolean getBoolean(String colName) {
353-
return readNumberValue(colName, NumberType.Boolean);
347+
return readNumberValue(colName, NumberConverter.NumberType.Boolean);
354348
}
355349

356350
@Override
357351
public BigInteger getBigInteger(String colName) {
358-
return readNumberValue(colName, NumberType.BigInteger);
352+
return readNumberValue(colName, NumberConverter.NumberType.BigInteger);
359353
}
360354

361355
@Override
362356
public BigDecimal getBigDecimal(String colName) {
363-
return readNumberValue(colName, NumberType.BigDecimal);
357+
return readNumberValue(colName, NumberConverter.NumberType.BigDecimal);
364358
}
365359

366360
@Override
@@ -733,19 +727,4 @@ public ClickHouseBitmap getClickHouseBitmap(int index) {
733727
public void close() throws Exception {
734728
input.close();
735729
}
736-
737-
private enum NumberType {
738-
Byte("byte"), Short("short"), Int("int"), Long("long"), BigInteger("BigInteger"), Float("float"),
739-
Double("double"), BigDecimal("BigDecimal"), Boolean("boolean");
740-
741-
private final String typeName;
742-
743-
NumberType(String typeName) {
744-
this.typeName = typeName;
745-
}
746-
747-
public String getTypeName() {
748-
return typeName;
749-
}
750-
}
751730
}

0 commit comments

Comments
 (0)