Skip to content

Commit 3012dba

Browse files
committed
added support of reading new JSON as string
1 parent 3667f7d commit 3012dba

File tree

6 files changed

+60
-30
lines changed

6 files changed

+60
-30
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1903,7 +1903,7 @@ public ClickHouseBinaryFormatReader newBinaryFormatReader(QueryResponse response
19031903
byteBufferPool);
19041904
break;
19051905
default:
1906-
throw new IllegalArgumentException("Unsupported format: " + response.getFormat());
1906+
throw new IllegalArgumentException("Binary readers doesn't support format: " + response.getFormat());
19071907
}
19081908
return reader;
19091909
}

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

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

33
import com.clickhouse.client.api.ClientException;
4+
import com.clickhouse.client.api.ClientSettings;
45
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
6+
import com.clickhouse.client.api.internal.MapUtils;
57
import com.clickhouse.client.api.metadata.TableSchema;
68
import com.clickhouse.client.api.query.NullValueException;
79
import com.clickhouse.client.api.query.POJOSetter;
@@ -32,8 +34,10 @@
3234
import java.time.temporal.ChronoUnit;
3335
import java.util.Collections;
3436
import java.util.HashMap;
37+
import java.util.HashSet;
3538
import java.util.List;
3639
import java.util.Map;
40+
import java.util.Set;
3741
import java.util.TimeZone;
3842
import java.util.UUID;
3943
import java.util.concurrent.ConcurrentHashMap;
@@ -71,7 +75,8 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer
7175
if (timeZone == null) {
7276
throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")");
7377
}
74-
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator);
78+
boolean jsonAsString = MapUtils.getFlag(querySettings.getAllSettings(), ClientSettings.SERVER_SETTING_PREFIX + "output_format_binary_write_json_as_string", false);
79+
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString);
7580
if (schema != null) {
7681
setSchema(schema);
7782
}

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

Lines changed: 10 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -45,16 +45,7 @@ public class BinaryStreamReader {
4545

4646
private final ByteBufferAllocator bufferAllocator;
4747

48-
/**
49-
* Creates a BinaryStreamReader instance that will use {@link DefaultByteBufferAllocator} to allocate buffers.
50-
*
51-
* @param input - source of raw data in a suitable format
52-
* @param timeZone - timezone to use for date and datetime values
53-
* @param log - logger
54-
*/
55-
BinaryStreamReader(InputStream input, TimeZone timeZone, Logger log) {
56-
this(input, timeZone, log, new DefaultByteBufferAllocator());
57-
}
48+
private final boolean jsonAsString;
5849

5950
/**
6051
* Createa a BinaryStreamReader instance that will use the provided buffer allocator.
@@ -64,11 +55,12 @@ public class BinaryStreamReader {
6455
* @param log - logger
6556
* @param bufferAllocator - byte buffer allocator
6657
*/
67-
BinaryStreamReader(InputStream input, TimeZone timeZone, Logger log, ByteBufferAllocator bufferAllocator) {
58+
BinaryStreamReader(InputStream input, TimeZone timeZone, Logger log, ByteBufferAllocator bufferAllocator, boolean jsonAsString) {
6859
this.log = log == null ? NOPLogger.NOP_LOGGER : log;
6960
this.timeZone = timeZone;
7061
this.input = input;
7162
this.bufferAllocator = bufferAllocator;
63+
this.jsonAsString = jsonAsString;
7264
}
7365

7466
/**
@@ -203,8 +195,13 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
203195
case Ring:
204196
return (T) readGeoRing();
205197

206-
// case JSON: // obsolete https://clickhouse.com/docs/en/sql-reference/data-types/json#displaying-json-column
207-
// case Object:
198+
case JSON: // experimental https://clickhouse.com/docs/en/sql-reference/data-types/newjson
199+
if (jsonAsString) {
200+
return (T) readString(input);
201+
} else {
202+
throw new RuntimeException("Reading JSON from binary is not implemented yet");
203+
}
204+
// case Object: // deprecated https://clickhouse.com/docs/en/sql-reference/data-types/object-data-type
208205
case Array:
209206
return convertArray(readArray(column), typeHint);
210207
case Map:

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

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -303,12 +303,8 @@ public Exception readError(ClassicHttpResponse httpResponse) {
303303
}
304304
}
305305
if (found) {
306-
int start = i;
307-
while (i < rBytes && buffer[i] != '\n') {
308-
i++;
309-
}
310306

311-
return new ServerException(serverCode, new String(buffer, start, i -start, StandardCharsets.UTF_8));
307+
return new ServerException(serverCode, new String(buffer, i, rBytes - i, StandardCharsets.UTF_8));
312308
}
313309
}
314310
}

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

Lines changed: 11 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -68,21 +68,24 @@ public static boolean getFlag(Map<String, String> map, String key) {
6868
throw new IllegalArgumentException("Invalid non-boolean value for the key '" + key + "': '" + val + "'");
6969
}
7070

71-
public static boolean getFlag(Map<String, String> map, String key, boolean defaultValue) {
72-
String val = map.get(key);
71+
public static boolean getFlag(Map<String, ?> map, String key, boolean defaultValue) {
72+
Object val = map.get(key);
7373
if (val == null) {
7474
return defaultValue;
7575
}
76-
if (val.equalsIgnoreCase("true")) {
77-
return true;
78-
} else if (val.equalsIgnoreCase("false")) {
79-
return false;
76+
if (val instanceof Boolean) {
77+
return (Boolean) val;
78+
} else if (val instanceof String) {
79+
String str = (String) val;
80+
if (str.equalsIgnoreCase("true") || str.equalsIgnoreCase("1")) {
81+
return true;
82+
} else if (str.equalsIgnoreCase("false") || str.equalsIgnoreCase("0")) {
83+
return false;
84+
}
8085
}
81-
8286
throw new IllegalArgumentException("Invalid non-boolean value for the key '" + key + "': '" + val + "'");
8387
}
8488

85-
8689
public static boolean getFlag(Map<String, ?> p1, Map<String, ?> p2, String key) {
8790
Object val = p1.get(key);
8891
if (val == null) {

client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java

Lines changed: 31 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
import com.clickhouse.client.api.DataTypeUtils;
1717
import com.clickhouse.client.api.ServerException;
1818
import com.clickhouse.client.api.command.CommandResponse;
19+
import com.clickhouse.client.api.command.CommandSettings;
1920
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
2021
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
2122
import com.clickhouse.client.api.enums.Protocol;
@@ -38,6 +39,7 @@
3839
import com.fasterxml.jackson.databind.JsonNode;
3940
import com.fasterxml.jackson.databind.MappingIterator;
4041
import com.fasterxml.jackson.databind.ObjectMapper;
42+
import org.apache.commons.lang3.StringEscapeUtils;
4143
import org.testng.Assert;
4244
import org.testng.annotations.AfterMethod;
4345
import org.testng.annotations.BeforeMethod;
@@ -123,7 +125,6 @@ public void setUp() {
123125
.compressClientRequest(false)
124126
.compressServerResponse(useServerCompression)
125127
.useHttpCompression(useHttpCompression)
126-
.useNewImplementation(true)
127128
.build();
128129

129130
delayForProfiler(0);
@@ -342,7 +343,7 @@ public void testQueryAllTableNames() {
342343
}
343344

344345
@Test(groups = {"integration"})
345-
public void testQueryJSON() throws ExecutionException, InterruptedException {
346+
public void testQueryJSONEachRow() throws ExecutionException, InterruptedException {
346347
Map<String, Object> datasetRecord = prepareSimpleDataSet();
347348
QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.JSONEachRow);
348349
Future<QueryResponse> response = client.query("SELECT * FROM " + DATASET_TABLE, settings);
@@ -1795,6 +1796,34 @@ public void testReadingBitmap() throws Exception {
17951796
}
17961797
}
17971798

1799+
@Test(groups = {"integration"})
1800+
public void testReadingJSONValues() throws Exception {
1801+
1802+
CommandSettings commandSettings = new CommandSettings();
1803+
commandSettings.serverSetting("allow_experimental_json_type", "1");
1804+
client.execute("DROP TABLE IF EXISTS test_json_values", commandSettings).get(1, TimeUnit.SECONDS);
1805+
client.execute("CREATE TABLE test_json_values (json JSON) ENGINE = MergeTree ORDER BY ()", commandSettings).get(1, TimeUnit.SECONDS);
1806+
client.execute("INSERT INTO test_json_values VALUES ('{\"a\" : {\"b\" : 42}, \"c\" : [1, 2, 3]}')", commandSettings).get(1, TimeUnit.SECONDS);
1807+
1808+
1809+
QuerySettings settings = new QuerySettings()
1810+
.serverSetting("allow_experimental_json_type", "1")
1811+
.setFormat(ClickHouseFormat.CSV);
1812+
try (QueryResponse resp = client.query("SELECT json FROM test_json_values", settings).get(1, TimeUnit.SECONDS)) {
1813+
BufferedReader reader = new BufferedReader(new InputStreamReader(resp.getInputStream()));
1814+
Assert.assertEquals(StringEscapeUtils.unescapeCsv(reader.lines().findFirst().get()), "{\"a\":{\"b\":\"42\"},\"c\":[\"1\",\"2\",\"3\"]}");
1815+
}
1816+
1817+
settings = new QuerySettings()
1818+
.serverSetting("allow_experimental_json_type", "1")
1819+
.serverSetting("output_format_binary_write_json_as_string", "1");
1820+
try (QueryResponse resp = client.query("SELECT json FROM test_json_values", settings).get(1, TimeUnit.SECONDS)) {
1821+
ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(resp);
1822+
Assert.assertNotNull(reader.next());
1823+
Assert.assertEquals(reader.getString(1), "{\"a\":{\"b\":\"42\"},\"c\":[\"1\",\"2\",\"3\"]}");
1824+
}
1825+
}
1826+
17981827
protected Client.Builder newClient() {
17991828
ClickHouseNode node = getServer(ClickHouseProtocol.HTTP);
18001829
return new Client.Builder()

0 commit comments

Comments
 (0)