From f411452220a6ffb927d9a8dbf09876b0bf4f6e3d Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Sun, 15 Feb 2026 15:59:13 -0800 Subject: [PATCH 1/8] Added getting arrays of objects and other types of arrays --- .../ClickHouseBinaryFormatReader.java | 20 ++ .../internal/AbstractBinaryFormatReader.java | 34 +- .../internal/BinaryReaderBackedRecord.java | 10 + .../internal/BinaryStreamReader.java | 15 + .../internal/MapBackedRecord.java | 36 +- .../client/api/query/GenericRecord.java | 20 ++ .../ClickHouseBinaryFormatReaderTest.java | 330 ++++++++++++++++++ .../api/data_formats/RowBinaryTest.java | 283 ++++++++++++++- 8 files changed, 736 insertions(+), 12 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index df6979412..fca9daccb 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -308,6 +308,16 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { */ String[] getStringArray(String colName); + /** + * Reads column with name {@code colName} as an array of objects. Works for any array element type + * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. + * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. + * + * @param colName - column name + * @return array of objects or null if value is null + */ + Object[] getObjectArray(String colName); + /** * Reads column with name `colName` as a string. * @@ -536,6 +546,16 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { String[] getStringArray(int index); + /** + * Reads column at the specified index as an array of objects. Works for any array element type + * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. + * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. + * + * @param index - column index (1-based) + * @return array of objects or null if value is null + */ + Object[] getObjectArray(int index); + Object[] getTuple(int index); Object[] getTuple(String colName); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 2ebf2ffa0..5a9e23fe8 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -562,6 +562,11 @@ public String[] getStringArray(String colName) { return getStringArray(schema.nameToColumnIndex(colName)); } + @Override + public Object[] getObjectArray(String colName) { + return getObjectArray(schema.nameToColumnIndex(colName)); + } + @Override public boolean hasValue(int colIndex) { return currentRecord[colIndex - 1] != null; @@ -817,15 +822,34 @@ public String[] getStringArray(int index) { if (value instanceof BinaryStreamReader.ArrayValue) { BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; int length = array.length; - if (!array.itemType.equals(String.class)) - throw new ClientException("Not A String type."); String[] values = new String[length]; - for (int i = 0; i < length; i++) { - values[i] = (String)((BinaryStreamReader.ArrayValue) value).get(i); + if (array.itemType.equals(String.class)) { + for (int i = 0; i < length; i++) { + values[i] = (String) array.get(i); + } + } else { + for (int i = 0; i < length; i++) { + Object item = array.get(i); + values[i] = item == null ? null : item.toString(); + } } return values; } - throw new ClientException("Not ArrayValue type."); + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getObjectArray(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java index e06b5225a..7faa8e0fe 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java @@ -175,6 +175,11 @@ public String[] getStringArray(String colName) { return reader.getStringArray(colName); } + @Override + public Object[] getObjectArray(String colName) { + return reader.getObjectArray(colName); + } + @Override public String getString(int index) { return reader.getString(index); @@ -335,6 +340,11 @@ public String[] getStringArray(int index) { return reader.getStringArray(index); } + @Override + public Object[] getObjectArray(int index) { + return reader.getObjectArray(index); + } + @Override public Object[] getTuple(int index) { return reader.getTuple(index); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java index d0ba7eb95..14023e2b4 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java @@ -752,6 +752,21 @@ public Object[] getArrayOfObjects() { return (Object[]) array; } } + + /** + * Returns array of objects, recursively converting nested ArrayValue elements to Object[]. + * This is useful for nested arrays (e.g. Array(Array(Int64))) where elements are ArrayValue instances. + * + * @return Object[] with nested ArrayValue elements converted to Object[] + */ + public Object[] toObjectArray() { + Object[] result = new Object[length]; + for (int i = 0; i < length; i++) { + Object item = get(i); + result[i] = (item instanceof ArrayValue) ? ((ArrayValue) item).toObjectArray() : item; + } + return result; + } } public static class EnumValue extends Number { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index 35d138846..1d6c5f39b 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -296,15 +296,20 @@ public String[] getStringArray(String colName) { if (value instanceof BinaryStreamReader.ArrayValue) { BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; int length = array.length; - if (!array.itemType.equals(String.class)) - throw new ClientException("Not A String type."); - String [] values = new String[length]; - for (int i = 0; i < length; i++) { - values[i] = (String)((BinaryStreamReader.ArrayValue) value).get(i); + String[] values = new String[length]; + if (array.itemType.equals(String.class)) { + for (int i = 0; i < length; i++) { + values[i] = (String) array.get(i); + } + } else { + for (int i = 0; i < length; i++) { + Object item = array.get(i); + values[i] = item == null ? null : item.toString(); + } } return values; } - throw new ClientException("Not ArrayValue type."); + throw new ClientException("Column is not of array type"); } @Override @@ -480,6 +485,25 @@ public String[] getStringArray(int index) { return getPrimitiveArray(schema.columnIndexToName(index)); } + @Override + public Object[] getObjectArray(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getObjectArray(int index) { + return getObjectArray(schema.columnIndexToName(index)); + } + @Override public Object[] getTuple(int index) { return readValue(index); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java index e50dc82ee..5d13844ac 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java @@ -250,6 +250,16 @@ public interface GenericRecord { String[] getStringArray(String colName); + /** + * Reads column with name `colName` as an array of objects. Works for any array element type + * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. + * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. + * + * @param colName - column name + * @return array of objects or null if value is null + */ + Object[] getObjectArray(String colName); + /** * Reads column with name `colName` as a string. * @@ -490,6 +500,16 @@ public interface GenericRecord { String[] getStringArray(int index); + /** + * Reads column at the specified index as an array of objects. Works for any array element type + * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. + * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. + * + * @param index - column index (1-based) + * @return array of objects or null if value is null + */ + Object[] getObjectArray(int index); + Object[] getTuple(int index); Object[] getTuple(String colName); diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReaderTest.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReaderTest.java index f6206a3a4..55f4a2318 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReaderTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReaderTest.java @@ -12,6 +12,8 @@ import java.io.InputStream; import java.math.BigDecimal; import java.math.BigInteger; +import java.time.LocalDateTime; +import java.time.ZonedDateTime; import java.util.Arrays; import java.util.TimeZone; import java.util.function.Consumer; @@ -246,4 +248,332 @@ public void testReadingArrays() throws Exception { Assert.assertEquals(reader.getLongArray("a5"), new long[] {1L, 2L}); } + + @Test + public void testGetObjectArray1D() throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + + String[] names = new String[]{"uint64_arr", "enum_arr", "dt_arr", "fstr_arr", "str_arr"}; + String[] types = new String[]{ + "Array(UInt64)", + "Array(Enum8('abc' = 1, 'cde' = 2))", + "Array(DateTime('UTC'))", + "Array(FixedString(4))", + "Array(String)" + }; + + BinaryStreamUtils.writeVarInt(out, names.length); + for (String name : names) { + BinaryStreamUtils.writeString(out, name); + } + for (String type : types) { + BinaryStreamUtils.writeString(out, type); + } + + // Array(UInt64): [100, 200] + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeUnsignedInt64(out, BigInteger.valueOf(100)); + BinaryStreamUtils.writeUnsignedInt64(out, BigInteger.valueOf(200)); + + // Array(Enum8('abc' = 1, 'cde' = 2)): [1, 2] + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeEnum8(out, (byte) 1); + BinaryStreamUtils.writeEnum8(out, (byte) 2); + + // Array(DateTime('UTC')): two timestamps + LocalDateTime dt1 = LocalDateTime.of(2030, 10, 9, 8, 7, 6); + LocalDateTime dt2 = LocalDateTime.of(2031, 10, 9, 8, 7, 6); + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeDateTime32(out, dt1, TimeZone.getTimeZone("UTC")); + BinaryStreamUtils.writeDateTime32(out, dt2, TimeZone.getTimeZone("UTC")); + + // Array(FixedString(4)): ["abcd", "efgh"] + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeFixedString(out, "abcd", 4); + BinaryStreamUtils.writeFixedString(out, "efgh", 4); + + // Array(String): ["hello", "world"] + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeString(out, "hello"); + BinaryStreamUtils.writeString(out, "world"); + + InputStream in = new ByteArrayInputStream(out.toByteArray()); + QuerySettings querySettings = new QuerySettings().setUseTimeZone("UTC"); + RowBinaryWithNamesAndTypesFormatReader reader = + new RowBinaryWithNamesAndTypesFormatReader(in, querySettings, new BinaryStreamReader.CachingByteBufferAllocator()); + reader.next(); + + // Test Array(UInt64) via getObjectArray + Object[] uint64Result = reader.getObjectArray("uint64_arr"); + Assert.assertNotNull(uint64Result); + Assert.assertEquals(uint64Result.length, 2); + Assert.assertEquals(uint64Result[0], BigInteger.valueOf(100)); + Assert.assertEquals(uint64Result[1], BigInteger.valueOf(200)); + + // Test Array(Enum8) via getObjectArray + Object[] enumResult = reader.getObjectArray("enum_arr"); + Assert.assertNotNull(enumResult); + Assert.assertEquals(enumResult.length, 2); + Assert.assertTrue(enumResult[0] instanceof BinaryStreamReader.EnumValue); + Assert.assertEquals(enumResult[0].toString(), "abc"); + Assert.assertEquals(enumResult[1].toString(), "cde"); + + // Test Array(Enum8) via getStringArray (sugar) + String[] enumStrings = reader.getStringArray("enum_arr"); + Assert.assertEquals(enumStrings, new String[]{"abc", "cde"}); + + // Test Array(DateTime) via getObjectArray + Object[] dtResult = reader.getObjectArray("dt_arr"); + Assert.assertNotNull(dtResult); + Assert.assertEquals(dtResult.length, 2); + Assert.assertTrue(dtResult[0] instanceof ZonedDateTime); + ZonedDateTime zdt1 = (ZonedDateTime) dtResult[0]; + ZonedDateTime zdt2 = (ZonedDateTime) dtResult[1]; + Assert.assertEquals(zdt1.toLocalDateTime(), dt1); + Assert.assertEquals(zdt2.toLocalDateTime(), dt2); + + // Test Array(FixedString) via getObjectArray + Object[] fstrResult = reader.getObjectArray("fstr_arr"); + Assert.assertNotNull(fstrResult); + Assert.assertEquals(fstrResult.length, 2); + Assert.assertEquals(fstrResult[0], "abcd"); + Assert.assertEquals(fstrResult[1], "efgh"); + + // Test Array(String) via getObjectArray + Object[] strResult = reader.getObjectArray("str_arr"); + Assert.assertNotNull(strResult); + Assert.assertEquals(strResult.length, 2); + Assert.assertEquals(strResult[0], "hello"); + Assert.assertEquals(strResult[1], "world"); + + // Also verify getObjectArray works for primitive-backed arrays too + // (int arrays are still returned as boxed objects) + } + + @Test + public void testGetObjectArray2D() throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + + String[] names = new String[]{"arr2d_int", "arr2d_str"}; + String[] types = new String[]{"Array(Array(Int64))", "Array(Array(String))"}; + + BinaryStreamUtils.writeVarInt(out, names.length); + for (String name : names) { + BinaryStreamUtils.writeString(out, name); + } + for (String type : types) { + BinaryStreamUtils.writeString(out, type); + } + + // Array(Array(Int64)): [[1, 2, 3], [4, 5]] + BinaryStreamUtils.writeVarInt(out, 2); // outer array length + BinaryStreamUtils.writeVarInt(out, 3); // inner[0] length + BinaryStreamUtils.writeInt64(out, 1L); + BinaryStreamUtils.writeInt64(out, 2L); + BinaryStreamUtils.writeInt64(out, 3L); + BinaryStreamUtils.writeVarInt(out, 2); // inner[1] length + BinaryStreamUtils.writeInt64(out, 4L); + BinaryStreamUtils.writeInt64(out, 5L); + + // Array(Array(String)): [["a", "b"], ["c"]] + BinaryStreamUtils.writeVarInt(out, 2); // outer array length + BinaryStreamUtils.writeVarInt(out, 2); // inner[0] length + BinaryStreamUtils.writeString(out, "a"); + BinaryStreamUtils.writeString(out, "b"); + BinaryStreamUtils.writeVarInt(out, 1); // inner[1] length + BinaryStreamUtils.writeString(out, "c"); + + InputStream in = new ByteArrayInputStream(out.toByteArray()); + QuerySettings querySettings = new QuerySettings().setUseTimeZone("UTC"); + RowBinaryWithNamesAndTypesFormatReader reader = + new RowBinaryWithNamesAndTypesFormatReader(in, querySettings, new BinaryStreamReader.CachingByteBufferAllocator()); + reader.next(); + + // Test 2D int array + Object[] arr2dInt = reader.getObjectArray("arr2d_int"); + Assert.assertNotNull(arr2dInt); + Assert.assertEquals(arr2dInt.length, 2); + + // Inner arrays should be Object[] (recursively converted) + Assert.assertTrue(arr2dInt[0] instanceof Object[]); + Assert.assertTrue(arr2dInt[1] instanceof Object[]); + + Object[] inner0 = (Object[]) arr2dInt[0]; + Object[] inner1 = (Object[]) arr2dInt[1]; + Assert.assertEquals(inner0.length, 3); + Assert.assertEquals(inner0[0], 1L); + Assert.assertEquals(inner0[1], 2L); + Assert.assertEquals(inner0[2], 3L); + Assert.assertEquals(inner1.length, 2); + Assert.assertEquals(inner1[0], 4L); + Assert.assertEquals(inner1[1], 5L); + + // Test 2D string array + Object[] arr2dStr = reader.getObjectArray("arr2d_str"); + Assert.assertNotNull(arr2dStr); + Assert.assertEquals(arr2dStr.length, 2); + Assert.assertTrue(arr2dStr[0] instanceof Object[]); + Assert.assertTrue(arr2dStr[1] instanceof Object[]); + + Object[] strInner0 = (Object[]) arr2dStr[0]; + Object[] strInner1 = (Object[]) arr2dStr[1]; + Assert.assertEquals(strInner0, new Object[]{"a", "b"}); + Assert.assertEquals(strInner1, new Object[]{"c"}); + } + + @Test + public void testGetObjectArray3D() throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + + String[] names = new String[]{"arr3d"}; + String[] types = new String[]{"Array(Array(Array(Int32)))"}; + + BinaryStreamUtils.writeVarInt(out, names.length); + for (String name : names) { + BinaryStreamUtils.writeString(out, name); + } + for (String type : types) { + BinaryStreamUtils.writeString(out, type); + } + + // Array(Array(Array(Int32))): [[[1, 2], [3]], [[4]]] + BinaryStreamUtils.writeVarInt(out, 2); // dim1 length = 2 + // dim1[0] = [[1, 2], [3]] + BinaryStreamUtils.writeVarInt(out, 2); // dim2 length = 2 + BinaryStreamUtils.writeVarInt(out, 2); // dim3 length = 2 + BinaryStreamUtils.writeInt32(out, 1); + BinaryStreamUtils.writeInt32(out, 2); + BinaryStreamUtils.writeVarInt(out, 1); // dim3 length = 1 + BinaryStreamUtils.writeInt32(out, 3); + // dim1[1] = [[4]] + BinaryStreamUtils.writeVarInt(out, 1); // dim2 length = 1 + BinaryStreamUtils.writeVarInt(out, 1); // dim3 length = 1 + BinaryStreamUtils.writeInt32(out, 4); + + InputStream in = new ByteArrayInputStream(out.toByteArray()); + QuerySettings querySettings = new QuerySettings().setUseTimeZone("UTC"); + RowBinaryWithNamesAndTypesFormatReader reader = + new RowBinaryWithNamesAndTypesFormatReader(in, querySettings, new BinaryStreamReader.CachingByteBufferAllocator()); + reader.next(); + + // Test 3D array: [[[1, 2], [3]], [[4]]] + Object[] arr3d = reader.getObjectArray("arr3d"); + Assert.assertNotNull(arr3d); + Assert.assertEquals(arr3d.length, 2); + + // dim1[0] = [[1, 2], [3]] + Assert.assertTrue(arr3d[0] instanceof Object[]); + Object[] dim1_0 = (Object[]) arr3d[0]; + Assert.assertEquals(dim1_0.length, 2); + + // dim1[0][0] = [1, 2] + Assert.assertTrue(dim1_0[0] instanceof Object[]); + Object[] dim2_0_0 = (Object[]) dim1_0[0]; + Assert.assertEquals(dim2_0_0.length, 2); + Assert.assertEquals(dim2_0_0[0], 1); + Assert.assertEquals(dim2_0_0[1], 2); + + // dim1[0][1] = [3] + Assert.assertTrue(dim1_0[1] instanceof Object[]); + Object[] dim2_0_1 = (Object[]) dim1_0[1]; + Assert.assertEquals(dim2_0_1.length, 1); + Assert.assertEquals(dim2_0_1[0], 3); + + // dim1[1] = [[4]] + Assert.assertTrue(arr3d[1] instanceof Object[]); + Object[] dim1_1 = (Object[]) arr3d[1]; + Assert.assertEquals(dim1_1.length, 1); + + Assert.assertTrue(dim1_1[0] instanceof Object[]); + Object[] dim2_1_0 = (Object[]) dim1_1[0]; + Assert.assertEquals(dim2_1_0.length, 1); + Assert.assertEquals(dim2_1_0[0], 4); + } + + @Test + public void testGetObjectArrayEmpty() throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + + String[] names = new String[]{"empty_arr"}; + String[] types = new String[]{"Array(Int32)"}; + + BinaryStreamUtils.writeVarInt(out, names.length); + for (String name : names) { + BinaryStreamUtils.writeString(out, name); + } + for (String type : types) { + BinaryStreamUtils.writeString(out, type); + } + + // Empty array + BinaryStreamUtils.writeVarInt(out, 0); + + InputStream in = new ByteArrayInputStream(out.toByteArray()); + QuerySettings querySettings = new QuerySettings().setUseTimeZone("UTC"); + RowBinaryWithNamesAndTypesFormatReader reader = + new RowBinaryWithNamesAndTypesFormatReader(in, querySettings, new BinaryStreamReader.CachingByteBufferAllocator()); + reader.next(); + + Object[] result = reader.getObjectArray("empty_arr"); + Assert.assertNotNull(result); + Assert.assertEquals(result.length, 0); + } + + @Test + public void testGetObjectArrayPrimitiveTypes() throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + + String[] names = new String[]{"int_arr", "bool_arr", "float_arr"}; + String[] types = new String[]{"Array(Int32)", "Array(Bool)", "Array(Float64)"}; + + BinaryStreamUtils.writeVarInt(out, names.length); + for (String name : names) { + BinaryStreamUtils.writeString(out, name); + } + for (String type : types) { + BinaryStreamUtils.writeString(out, type); + } + + // Array(Int32): [10, 20, 30] + BinaryStreamUtils.writeVarInt(out, 3); + BinaryStreamUtils.writeInt32(out, 10); + BinaryStreamUtils.writeInt32(out, 20); + BinaryStreamUtils.writeInt32(out, 30); + + // Array(Bool): [true, false] + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeBoolean(out, true); + BinaryStreamUtils.writeBoolean(out, false); + + // Array(Float64): [1.5, 2.5] + BinaryStreamUtils.writeVarInt(out, 2); + BinaryStreamUtils.writeFloat64(out, 1.5); + BinaryStreamUtils.writeFloat64(out, 2.5); + + InputStream in = new ByteArrayInputStream(out.toByteArray()); + QuerySettings querySettings = new QuerySettings().setUseTimeZone("UTC"); + RowBinaryWithNamesAndTypesFormatReader reader = + new RowBinaryWithNamesAndTypesFormatReader(in, querySettings, new BinaryStreamReader.CachingByteBufferAllocator()); + reader.next(); + + // getObjectArray should work for primitive-backed arrays too (auto-boxes) + Object[] intResult = reader.getObjectArray("int_arr"); + Assert.assertNotNull(intResult); + Assert.assertEquals(intResult.length, 3); + Assert.assertEquals(intResult[0], 10); + Assert.assertEquals(intResult[1], 20); + Assert.assertEquals(intResult[2], 30); + + Object[] boolResult = reader.getObjectArray("bool_arr"); + Assert.assertNotNull(boolResult); + Assert.assertEquals(boolResult.length, 2); + Assert.assertEquals(boolResult[0], true); + Assert.assertEquals(boolResult[1], false); + + Object[] floatResult = reader.getObjectArray("float_arr"); + Assert.assertNotNull(floatResult); + Assert.assertEquals(floatResult.length, 2); + Assert.assertEquals(floatResult[0], 1.5); + Assert.assertEquals(floatResult[1], 2.5); + } } \ No newline at end of file diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java index d5acebe10..b7e097080 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java @@ -5,15 +5,21 @@ import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.ClickHouseServerForTest; import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.data.ClickHouseFormat; import lombok.Data; import org.testng.Assert; import org.testng.annotations.Test; +import java.math.BigInteger; +import java.time.ZonedDateTime; import java.util.Collections; import java.util.List; -import java.util.Random; @Test(groups = {"integration"}) public class RowBinaryTest extends BaseIntegrationTest { @@ -63,6 +69,281 @@ void testDefaultWithFunction() { } } + @Test(groups = {"integration"}) + void testGetObjectArray1D() { + final String table = "test_get_object_array_1d"; + try (Client client = newClient().build()) { + client.execute("DROP TABLE IF EXISTS " + table); + client.execute("CREATE TABLE " + table + " (" + + "uint64_arr Array(UInt64), " + + "enum_arr Array(Enum8('abc' = 1, 'cde' = 2)), " + + "dt_arr Array(DateTime('UTC')), " + + "fstr_arr Array(FixedString(4)), " + + "str_arr Array(String)" + + ") ENGINE = MergeTree() ORDER BY tuple()"); + + client.execute("INSERT INTO " + table + " VALUES (" + + "[100, 200, 18000044073709551615], " + + "['abc', 'cde'], " + + "['2030-10-09 08:07:06', '2031-10-09 08:07:06'], " + + "['abcd', 'efgh'], " + + "['hello', 'world'])"); + + QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.RowBinary); + TableSchema schema = client.getTableSchema(table); + try (QueryResponse response = client.query("SELECT * FROM " + table, settings).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response, schema); + reader.next(); + + // Array(UInt64) -> BigInteger elements + Object[] uint64Arr = reader.getObjectArray("uint64_arr"); + Assert.assertNotNull(uint64Arr); + Assert.assertEquals(uint64Arr.length, 3); + Assert.assertEquals(uint64Arr[0], BigInteger.valueOf(100)); + Assert.assertEquals(uint64Arr[1], BigInteger.valueOf(200)); + Assert.assertEquals(uint64Arr[2], new BigInteger("18000044073709551615")); + + // Array(Enum8) -> EnumValue elements via getObjectArray + Object[] enumArr = reader.getObjectArray("enum_arr"); + Assert.assertNotNull(enumArr); + Assert.assertEquals(enumArr.length, 2); + Assert.assertTrue(enumArr[0] instanceof BinaryStreamReader.EnumValue); + Assert.assertEquals(enumArr[0].toString(), "abc"); + Assert.assertEquals(enumArr[1].toString(), "cde"); + + // Array(Enum8) -> String[] via getStringArray (sugar) + String[] enumStrings = reader.getStringArray("enum_arr"); + Assert.assertEquals(enumStrings, new String[]{"abc", "cde"}); + + // Array(DateTime) -> ZonedDateTime elements + Object[] dtArr = reader.getObjectArray("dt_arr"); + Assert.assertNotNull(dtArr); + Assert.assertEquals(dtArr.length, 2); + Assert.assertTrue(dtArr[0] instanceof ZonedDateTime); + Assert.assertTrue(dtArr[1] instanceof ZonedDateTime); + ZonedDateTime zdt1 = (ZonedDateTime) dtArr[0]; + ZonedDateTime zdt2 = (ZonedDateTime) dtArr[1]; + Assert.assertEquals(zdt1.getYear(), 2030); + Assert.assertEquals(zdt1.getMonthValue(), 10); + Assert.assertEquals(zdt1.getDayOfMonth(), 9); + Assert.assertEquals(zdt2.getYear(), 2031); + + // Array(FixedString(4)) -> String elements + Object[] fstrArr = reader.getObjectArray("fstr_arr"); + Assert.assertNotNull(fstrArr); + Assert.assertEquals(fstrArr.length, 2); + Assert.assertEquals(fstrArr[0], "abcd"); + Assert.assertEquals(fstrArr[1], "efgh"); + + // Array(String) -> String elements + Object[] strArr = reader.getObjectArray("str_arr"); + Assert.assertNotNull(strArr); + Assert.assertEquals(strArr[0], "hello"); + Assert.assertEquals(strArr[1], "world"); + + // getStringArray should also work for FixedString arrays + String[] fstrStrings = reader.getStringArray("fstr_arr"); + Assert.assertEquals(fstrStrings, new String[]{"abcd", "efgh"}); + + Assert.assertNull(reader.next(), "Expected only one row"); + } + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + @Test(groups = {"integration"}) + void testGetObjectArray2D() { + final String table = "test_get_object_array_2d"; + try (Client client = newClient().build()) { + client.execute("DROP TABLE IF EXISTS " + table); + client.execute("CREATE TABLE " + table + " (" + + "arr2d_int Array(Array(Int64)), " + + "arr2d_str Array(Array(String))" + + ") ENGINE = MergeTree() ORDER BY tuple()"); + + client.execute("INSERT INTO " + table + " VALUES (" + + "[[1, 2, 3], [4, 5]], " + + "[['hello', 'world'], ['foo']])"); + + QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.RowBinary); + TableSchema schema = client.getTableSchema(table); + try (QueryResponse response = client.query("SELECT * FROM " + table, settings).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response, schema); + reader.next(); + + // Array(Array(Int64)) -> nested Object[] + Object[] arr2dInt = reader.getObjectArray("arr2d_int"); + Assert.assertNotNull(arr2dInt); + Assert.assertEquals(arr2dInt.length, 2); + Assert.assertTrue(arr2dInt[0] instanceof Object[]); + Assert.assertTrue(arr2dInt[1] instanceof Object[]); + + Object[] inner0 = (Object[]) arr2dInt[0]; + Assert.assertEquals(inner0.length, 3); + Assert.assertEquals(inner0[0], 1L); + Assert.assertEquals(inner0[1], 2L); + Assert.assertEquals(inner0[2], 3L); + + Object[] inner1 = (Object[]) arr2dInt[1]; + Assert.assertEquals(inner1.length, 2); + Assert.assertEquals(inner1[0], 4L); + Assert.assertEquals(inner1[1], 5L); + + // Array(Array(String)) -> nested Object[] + Object[] arr2dStr = reader.getObjectArray("arr2d_str"); + Assert.assertNotNull(arr2dStr); + Assert.assertEquals(arr2dStr.length, 2); + + Object[] strInner0 = (Object[]) arr2dStr[0]; + Assert.assertEquals(strInner0, new Object[]{"hello", "world"}); + + Object[] strInner1 = (Object[]) arr2dStr[1]; + Assert.assertEquals(strInner1, new Object[]{"foo"}); + + Assert.assertNull(reader.next(), "Expected only one row"); + } + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + @Test(groups = {"integration"}) + void testGetObjectArray3D() { + final String table = "test_get_object_array_3d"; + try (Client client = newClient().build()) { + client.execute("DROP TABLE IF EXISTS " + table); + client.execute("CREATE TABLE " + table + " (" + + "arr3d Array(Array(Array(Int32)))" + + ") ENGINE = MergeTree() ORDER BY tuple()"); + + client.execute("INSERT INTO " + table + " VALUES (" + + "[[[1, 2], [3]], [[4, 5, 6]]])"); + + QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.RowBinary); + TableSchema schema = client.getTableSchema(table); + try (QueryResponse response = client.query("SELECT * FROM " + table, settings).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response, schema); + reader.next(); + + // Array(Array(Array(Int32))) -> 3-level nested Object[] + Object[] arr3d = reader.getObjectArray("arr3d"); + Assert.assertNotNull(arr3d); + Assert.assertEquals(arr3d.length, 2); + + // dim1[0] = [[1, 2], [3]] + Assert.assertTrue(arr3d[0] instanceof Object[]); + Object[] dim1_0 = (Object[]) arr3d[0]; + Assert.assertEquals(dim1_0.length, 2); + + Assert.assertTrue(dim1_0[0] instanceof Object[]); + Object[] dim2_0_0 = (Object[]) dim1_0[0]; + Assert.assertEquals(dim2_0_0.length, 2); + Assert.assertEquals(dim2_0_0[0], 1); + Assert.assertEquals(dim2_0_0[1], 2); + + Assert.assertTrue(dim1_0[1] instanceof Object[]); + Object[] dim2_0_1 = (Object[]) dim1_0[1]; + Assert.assertEquals(dim2_0_1.length, 1); + Assert.assertEquals(dim2_0_1[0], 3); + + // dim1[1] = [[4, 5, 6]] + Assert.assertTrue(arr3d[1] instanceof Object[]); + Object[] dim1_1 = (Object[]) arr3d[1]; + Assert.assertEquals(dim1_1.length, 1); + + Assert.assertTrue(dim1_1[0] instanceof Object[]); + Object[] dim2_1_0 = (Object[]) dim1_1[0]; + Assert.assertEquals(dim2_1_0.length, 3); + Assert.assertEquals(dim2_1_0[0], 4); + Assert.assertEquals(dim2_1_0[1], 5); + Assert.assertEquals(dim2_1_0[2], 6); + + Assert.assertNull(reader.next(), "Expected only one row"); + } + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + @Test(groups = {"integration"}) + void testGetObjectArrayWithEmptyArrays() { + final String table = "test_get_object_array_empty"; + try (Client client = newClient().build()) { + client.execute("DROP TABLE IF EXISTS " + table); + client.execute("CREATE TABLE " + table + " (" + + "empty_arr Array(Int32), " + + "empty_2d Array(Array(String))" + + ") ENGINE = MergeTree() ORDER BY tuple()"); + + client.execute("INSERT INTO " + table + " VALUES ([], [])"); + + QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.RowBinary); + TableSchema schema = client.getTableSchema(table); + try (QueryResponse response = client.query("SELECT * FROM " + table, settings).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response, schema); + reader.next(); + + Object[] emptyArr = reader.getObjectArray("empty_arr"); + Assert.assertNotNull(emptyArr); + Assert.assertEquals(emptyArr.length, 0); + + Object[] empty2d = reader.getObjectArray("empty_2d"); + Assert.assertNotNull(empty2d); + Assert.assertEquals(empty2d.length, 0); + + Assert.assertNull(reader.next(), "Expected only one row"); + } + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + @Test(groups = {"integration"}) + void testGetObjectArrayMultipleRows() { + final String table = "test_get_object_array_multi_row"; + try (Client client = newClient().build()) { + client.execute("DROP TABLE IF EXISTS " + table); + client.execute("CREATE TABLE " + table + " (" + + "id UInt32, " + + "arr Array(UInt64)" + + ") ENGINE = MergeTree() ORDER BY id"); + + client.execute("INSERT INTO " + table + " VALUES " + + "(1, [100, 200]), " + + "(2, [300]), " + + "(3, [])"); + + QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.RowBinary); + TableSchema schema = client.getTableSchema(table); + try (QueryResponse response = client.query("SELECT * FROM " + table + " ORDER BY id", settings).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response, schema); + + // Row 1 + reader.next(); + Object[] arr1 = reader.getObjectArray("arr"); + Assert.assertEquals(arr1.length, 2); + Assert.assertEquals(arr1[0], BigInteger.valueOf(100)); + Assert.assertEquals(arr1[1], BigInteger.valueOf(200)); + + // Row 2 + reader.next(); + Object[] arr2 = reader.getObjectArray("arr"); + Assert.assertEquals(arr2.length, 1); + Assert.assertEquals(arr2[0], BigInteger.valueOf(300)); + + // Row 3 + reader.next(); + Object[] arr3 = reader.getObjectArray("arr"); + Assert.assertEquals(arr3.length, 0); + + Assert.assertNull(reader.next(), "Expected only three rows"); + } + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + @Data public static class DefaultWithFunctionPojo { private String name; From f8d2099b1e18bcc94e69de19227e4d363105ebb6 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Sun, 15 Feb 2026 16:06:18 -0800 Subject: [PATCH 2/8] added tests --- .../client/datatypes/DataTypeTests.java | 196 ++++++++++++++++++ 1 file changed, 196 insertions(+) diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java index 52347c729..ede5fc685 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -1278,6 +1278,202 @@ public Object[][] testNestedArrays_dp() { }; } + @Test(groups = {"integration"}) + public void testGetObjectArrayMethods() throws Exception { + final String table = "test_get_object_array_methods"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute("CREATE TABLE " + table + " (" + + "rowId Int32, " + + "uint64_arr Array(UInt64), " + + "enum_arr Array(Enum8('abc' = 1, 'cde' = 2, 'xyz' = 3)), " + + "dt_arr Array(DateTime('UTC')), " + + "fstr_arr Array(FixedString(4)), " + + "str_arr Array(String), " + + "int_arr Array(Int32), " + + "arr2d Array(Array(Int64)), " + + "arr3d Array(Array(Array(Int32)))" + + ") Engine = MergeTree ORDER BY rowId").get(); + + client.execute("INSERT INTO " + table + " VALUES " + + "(1, " + + "[100, 200, 18000044073709551615], " + + "['abc', 'cde'], " + + "['2030-10-09 08:07:06', '2031-10-09 08:07:06'], " + + "['abcd', 'efgh'], " + + "['hello', 'world'], " + + "[10, 20, 30], " + + "[[1, 2, 3], [4, 5]], " + + "[[[1, 2], [3]], [[4, 5, 6]]]" + + "), " + + "(2, " + + "[], " + + "['xyz'], " + + "[], " + + "[], " + + "[], " + + "[], " + + "[[]], " + + "[[[]]]" + + ")").get(); + + List records = client.queryAll("SELECT * FROM " + table + " ORDER BY rowId"); + Assert.assertEquals(records.size(), 2); + + // --- Row 1: non-empty arrays --- + GenericRecord row1 = records.get(0); + Assert.assertEquals(row1.getInteger("rowId"), 1); + + // Array(UInt64) -> getObjectArray returns BigInteger[] + Object[] uint64Arr = row1.getObjectArray("uint64_arr"); + Assert.assertNotNull(uint64Arr); + Assert.assertEquals(uint64Arr.length, 3); + Assert.assertTrue(uint64Arr[0] instanceof java.math.BigInteger); + Assert.assertEquals(uint64Arr[0], java.math.BigInteger.valueOf(100)); + Assert.assertEquals(uint64Arr[1], java.math.BigInteger.valueOf(200)); + Assert.assertEquals(uint64Arr[2], new java.math.BigInteger("18000044073709551615")); + + // Array(UInt64) -> getStringArray converts via toString() + String[] uint64Strings = row1.getStringArray("uint64_arr"); + Assert.assertEquals(uint64Strings, new String[]{"100", "200", "18000044073709551615"}); + + // Array(Enum8) -> getObjectArray returns EnumValue[] + Object[] enumArr = row1.getObjectArray("enum_arr"); + Assert.assertNotNull(enumArr); + Assert.assertEquals(enumArr.length, 2); + Assert.assertTrue(enumArr[0] instanceof BinaryStreamReader.EnumValue); + Assert.assertEquals(enumArr[0].toString(), "abc"); + Assert.assertEquals(enumArr[1].toString(), "cde"); + + // Array(Enum8) -> getStringArray returns enum names + String[] enumStrings = row1.getStringArray("enum_arr"); + Assert.assertEquals(enumStrings, new String[]{"abc", "cde"}); + + // Array(DateTime) -> getObjectArray returns ZonedDateTime[] + Object[] dtArr = row1.getObjectArray("dt_arr"); + Assert.assertNotNull(dtArr); + Assert.assertEquals(dtArr.length, 2); + Assert.assertTrue(dtArr[0] instanceof java.time.ZonedDateTime); + java.time.ZonedDateTime zdt1 = (java.time.ZonedDateTime) dtArr[0]; + java.time.ZonedDateTime zdt2 = (java.time.ZonedDateTime) dtArr[1]; + Assert.assertEquals(zdt1.getYear(), 2030); + Assert.assertEquals(zdt1.getMonthValue(), 10); + Assert.assertEquals(zdt1.getDayOfMonth(), 9); + Assert.assertEquals(zdt1.getHour(), 8); + Assert.assertEquals(zdt1.getMinute(), 7); + Assert.assertEquals(zdt1.getSecond(), 6); + Assert.assertEquals(zdt2.getYear(), 2031); + + // Array(FixedString) -> getObjectArray returns String[] + Object[] fstrArr = row1.getObjectArray("fstr_arr"); + Assert.assertNotNull(fstrArr); + Assert.assertEquals(fstrArr.length, 2); + Assert.assertEquals(fstrArr[0], "abcd"); + Assert.assertEquals(fstrArr[1], "efgh"); + + // Array(FixedString) -> getStringArray + String[] fstrStrings = row1.getStringArray("fstr_arr"); + Assert.assertEquals(fstrStrings, new String[]{"abcd", "efgh"}); + + // Array(String) -> getObjectArray returns String[] + Object[] strArr = row1.getObjectArray("str_arr"); + Assert.assertNotNull(strArr); + Assert.assertEquals(strArr[0], "hello"); + Assert.assertEquals(strArr[1], "world"); + + // Array(Int32) -> getObjectArray returns boxed Integer[] + Object[] intArr = row1.getObjectArray("int_arr"); + Assert.assertNotNull(intArr); + Assert.assertEquals(intArr.length, 3); + Assert.assertEquals(intArr[0], 10); + Assert.assertEquals(intArr[1], 20); + Assert.assertEquals(intArr[2], 30); + + // Array(Array(Int64)) 2D -> getObjectArray returns nested Object[] + Object[] arr2d = row1.getObjectArray("arr2d"); + Assert.assertNotNull(arr2d); + Assert.assertEquals(arr2d.length, 2); + Assert.assertTrue(arr2d[0] instanceof Object[]); + Assert.assertTrue(arr2d[1] instanceof Object[]); + Object[] inner2d_0 = (Object[]) arr2d[0]; + Assert.assertEquals(inner2d_0.length, 3); + Assert.assertEquals(inner2d_0[0], 1L); + Assert.assertEquals(inner2d_0[1], 2L); + Assert.assertEquals(inner2d_0[2], 3L); + Object[] inner2d_1 = (Object[]) arr2d[1]; + Assert.assertEquals(inner2d_1.length, 2); + Assert.assertEquals(inner2d_1[0], 4L); + Assert.assertEquals(inner2d_1[1], 5L); + + // Array(Array(Array(Int32))) 3D -> getObjectArray returns 3-level nested Object[] + Object[] arr3d = row1.getObjectArray("arr3d"); + Assert.assertNotNull(arr3d); + Assert.assertEquals(arr3d.length, 2); + + // [[[1, 2], [3]], [[4, 5, 6]]] + Object[] dim1_0 = (Object[]) arr3d[0]; + Assert.assertEquals(dim1_0.length, 2); + Object[] dim2_0_0 = (Object[]) dim1_0[0]; + Assert.assertEquals(dim2_0_0.length, 2); + Assert.assertEquals(dim2_0_0[0], 1); + Assert.assertEquals(dim2_0_0[1], 2); + Object[] dim2_0_1 = (Object[]) dim1_0[1]; + Assert.assertEquals(dim2_0_1.length, 1); + Assert.assertEquals(dim2_0_1[0], 3); + + Object[] dim1_1 = (Object[]) arr3d[1]; + Assert.assertEquals(dim1_1.length, 1); + Object[] dim2_1_0 = (Object[]) dim1_1[0]; + Assert.assertEquals(dim2_1_0.length, 3); + Assert.assertEquals(dim2_1_0[0], 4); + Assert.assertEquals(dim2_1_0[1], 5); + Assert.assertEquals(dim2_1_0[2], 6); + + // --- Row 2: edge cases (empty arrays, single elements) --- + GenericRecord row2 = records.get(1); + Assert.assertEquals(row2.getInteger("rowId"), 2); + + // Empty arrays + Object[] emptyUint64 = row2.getObjectArray("uint64_arr"); + Assert.assertNotNull(emptyUint64); + Assert.assertEquals(emptyUint64.length, 0); + + Object[] emptyDt = row2.getObjectArray("dt_arr"); + Assert.assertNotNull(emptyDt); + Assert.assertEquals(emptyDt.length, 0); + + Object[] emptyStr = row2.getObjectArray("str_arr"); + Assert.assertNotNull(emptyStr); + Assert.assertEquals(emptyStr.length, 0); + + Object[] emptyInt = row2.getObjectArray("int_arr"); + Assert.assertNotNull(emptyInt); + Assert.assertEquals(emptyInt.length, 0); + + // Single-element enum array + Object[] singleEnum = row2.getObjectArray("enum_arr"); + Assert.assertEquals(singleEnum.length, 1); + Assert.assertEquals(singleEnum[0].toString(), "xyz"); + + String[] singleEnumStr = row2.getStringArray("enum_arr"); + Assert.assertEquals(singleEnumStr, new String[]{"xyz"}); + + // 2D with inner empty: [[]] + Object[] arr2dEmpty = row2.getObjectArray("arr2d"); + Assert.assertNotNull(arr2dEmpty); + Assert.assertEquals(arr2dEmpty.length, 1); + Assert.assertTrue(arr2dEmpty[0] instanceof Object[]); + Assert.assertEquals(((Object[]) arr2dEmpty[0]).length, 0); + + // 3D with inner empty: [[[]]] + Object[] arr3dEmpty = row2.getObjectArray("arr3d"); + Assert.assertNotNull(arr3dEmpty); + Assert.assertEquals(arr3dEmpty.length, 1); + Object[] arr3dInner = (Object[]) arr3dEmpty[0]; + Assert.assertEquals(arr3dInner.length, 1); + Assert.assertTrue(arr3dInner[0] instanceof Object[]); + Assert.assertEquals(((Object[]) arr3dInner[0]).length, 0); + } + public static String tableDefinition(String table, String... columns) { StringBuilder sb = new StringBuilder(); sb.append("CREATE TABLE " + table + " ( "); From 3214f6d5f5fde90263f62042a69c5779ad9c8a10 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Feb 2026 09:44:32 -0800 Subject: [PATCH 3/8] Corrected java doc --- .../ClickHouseBinaryFormatReader.java | 131 ++++++++++-------- 1 file changed, 76 insertions(+), 55 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index fca9daccb..e6bb61a5c 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -240,81 +240,72 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName); /** - * Reads column with name `colName` as a string. - * + * @see #getList(int) * @param colName - column name - * @return + * @return list of values, or {@code null} if the value is null */ List getList(String colName); /** - * Reads column with name `colName` as a string. - * + * @see #getByteArray(int) * @param colName - column name - * @return + * @return array of bytes, or {@code null} if the value is null */ byte[] getByteArray(String colName); /** - * Reads column with name `colName` as a string. - * + * @see #getIntArray(int) * @param colName - column name - * @return + * @return array of int values, or {@code null} if the value is null */ int[] getIntArray(String colName); /** - * Reads column with name `colName` as a string. - * + * @see #getLongArray(int) * @param colName - column name - * @return + * @return array of long values, or {@code null} if the value is null */ long[] getLongArray(String colName); /** - * Reads column with name `colName` as a string. - * + * @see #getFloatArray(int) * @param colName - column name - * @return + * @return array of float values, or {@code null} if the value is null */ float[] getFloatArray(String colName); /** - * Reads column with name `colName` as a string. - * + * @see #getDoubleArray(int) * @param colName - column name - * @return + * @return array of double values, or {@code null} if the value is null */ double[] getDoubleArray(String colName); /** - * - * @param colName - * @return + * @see #getBooleanArray(int) + * @param colName - column name + * @return array of boolean values, or {@code null} if the value is null */ boolean[] getBooleanArray(String colName); /** - * - * @param colName - * @return + * @see #getShortArray(int) + * @param colName - column name + * @return array of short values, or {@code null} if the value is null */ short[] getShortArray(String colName); /** - * - * @param colName - * @return + * @see #getStringArray(int) + * @param colName - column name + * @return array of string values, or {@code null} if the value is null */ String[] getStringArray(String colName); /** - * Reads column with name {@code colName} as an array of objects. Works for any array element type - * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. - * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. - * + * @see #getObjectArray(int) * @param colName - column name - * @return array of objects or null if value is null + * @return array of objects, or {@code null} if the value is null */ Object[] getObjectArray(String colName); @@ -493,66 +484,96 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index); /** - * Reads column with name `colName` as a string. + * Returns the value of the specified column as a {@link List}. Suitable for reading Array columns of any type. + *

For nested arrays (e.g. {@code Array(Array(Int64))}), returns a {@code List>}. + * For nullable arrays (e.g. {@code Array(Nullable(Int32))}), list elements may be {@code null}.

* - * @param index - column name - * @return + * @param index - column index (1-based) + * @return list of values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the column is not an array type */ List getList(int index); /** - * Reads column with name `colName` as a string. + * Returns the value of the specified column as a {@code byte[]}. Suitable for 1D Array columns only. * - * @param index - column name - * @return + * @param index - column index (1-based) + * @return array of bytes, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a byte array */ byte[] getByteArray(int index); /** - * Reads column with name `colName` as a string. + * Returns the value of the specified column as an {@code int[]}. Suitable for 1D Array columns only. * - * @param index - column name - * @return + * @param index - column index (1-based) + * @return array of int values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to an int array */ int[] getIntArray(int index); /** - * Reads column with name `colName` as a string. + * Returns the value of the specified column as a {@code long[]}. Suitable for 1D Array columns only. * - * @param index - column name - * @return + * @param index - column index (1-based) + * @return array of long values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a long array */ long[] getLongArray(int index); /** - * Reads column with name `colName` as a string. + * Returns the value of the specified column as a {@code float[]}. Suitable for 1D Array columns only. * - * @param index - column name - * @return + * @param index - column index (1-based) + * @return array of float values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a float array */ float[] getFloatArray(int index); /** - * Reads column with name `colName` as a string. + * Returns the value of the specified column as a {@code double[]}. Suitable for 1D Array columns only. * - * @param index - column name - * @return + * @param index - column index (1-based) + * @return array of double values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a double array */ double[] getDoubleArray(int index); + /** + * Returns the value of the specified column as a {@code boolean[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of boolean values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a boolean array + */ boolean[] getBooleanArray(int index); - short [] getShortArray(int index); + /** + * Returns the value of the specified column as a {@code short[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of short values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a short array + */ + short[] getShortArray(int index); + /** + * Returns the value of the specified column as a {@code String[]}. Suitable for 1D Array columns only. + * Cannot be used for none string element types. + * + * @param index - column index (1-based) + * @return array of string values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the column is not an array type + */ String[] getStringArray(int index); /** - * Reads column at the specified index as an array of objects. Works for any array element type - * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. - * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. + * Returns the value of the specified column as an {@code Object[]}. Suitable for multidimensional Array columns. + * Nested arrays are recursively converted to {@code Object[]}. * * @param index - column index (1-based) - * @return array of objects or null if value is null + * @return array of objects, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the column is not an array type */ Object[] getObjectArray(int index); From ac3f60d8f36d622c1686f303ccf7a3d9b3c233dd Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Feb 2026 09:59:51 -0800 Subject: [PATCH 4/8] Added getArray method --- .../ClickHouseBinaryFormatReader.java | 19 +++++++++++ .../internal/AbstractBinaryFormatReader.java | 33 ++++++++++++------- 2 files changed, 41 insertions(+), 11 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index e6bb61a5c..55950809d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -309,6 +309,14 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { */ Object[] getObjectArray(String colName); + /** + * @see #getArray(int) + * @param colName - column name + * @return Object - array value for the column. + * @throws com.clickhouse.client.api.ClientException if the column is not an array type + */ + Object getArray(String colName); + /** * Reads column with name `colName` as a string. * @@ -577,6 +585,17 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { */ Object[] getObjectArray(int index); + /** + * Returns reference to an array value of corresponding column. This method works for + * any multidimensional array values. However, it requires type cast so check column type. + * This method doesn't do a conversion between types. + * + * @param index column index + * @return Object - array value for the column. + * @throws com.clickhouse.client.api.ClientException if the column is not an array type + */ + Object getArray(int index); + Object[] getTuple(int index); Object[] getTuple(String colName); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 5a9e23fe8..09ef8fe66 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -567,6 +567,11 @@ public Object[] getObjectArray(String colName) { return getObjectArray(schema.nameToColumnIndex(colName)); } + @Override + public Object getArray(String colName) { + return getArray(schema.nameToColumnIndex(colName)); + } + @Override public boolean hasValue(int colIndex) { return currentRecord[colIndex - 1] != null; @@ -821,19 +826,11 @@ public String[] getStringArray(int index) { } if (value instanceof BinaryStreamReader.ArrayValue) { BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - int length = array.length; - String[] values = new String[length]; - if (array.itemType.equals(String.class)) { - for (int i = 0; i < length; i++) { - values[i] = (String) array.get(i); - } + if (array.itemType == String.class) { + return (String[]) array.getArray(); } else { - for (int i = 0; i < length; i++) { - Object item = array.get(i); - values[i] = item == null ? null : item.toString(); - } + throw new ClientException("Not an array of strings"); } - return values; } throw new ClientException("Column is not of array type"); } @@ -852,6 +849,20 @@ public Object[] getObjectArray(int index) { throw new ClientException("Column is not of array type"); } + @Override + public Object getArray(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).getArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); + } + @Override public Object[] getTuple(int index) { return readValue(index); From 076d6c58b46ada278461152362038e70f2bde62c Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Feb 2026 12:07:43 -0800 Subject: [PATCH 5/8] Updated tests and returned checking that array is of Strings --- .../ClickHouseBinaryFormatReader.java | 25 +++---------------- .../internal/AbstractBinaryFormatReader.java | 22 +++------------- .../internal/BinaryStreamReader.java | 6 +++++ .../api/data_formats/RowBinaryTest.java | 12 ++++++--- 4 files changed, 22 insertions(+), 43 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index 55950809d..19c8b08e9 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -307,15 +307,7 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { * @param colName - column name * @return array of objects, or {@code null} if the value is null */ - Object[] getObjectArray(String colName); - - /** - * @see #getArray(int) - * @param colName - column name - * @return Object - array value for the column. - * @throws com.clickhouse.client.api.ClientException if the column is not an array type - */ - Object getArray(String colName); + T getObjectArray(String colName); /** * Reads column with name `colName` as a string. @@ -578,23 +570,14 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { /** * Returns the value of the specified column as an {@code Object[]}. Suitable for multidimensional Array columns. * Nested arrays are recursively converted to {@code Object[]}. + * Note: result is not cached so avoid repetitive calls on same column. * + * @param - type of array like {@code Object[], Integer[][]} * @param index - column index (1-based) * @return array of objects, or {@code null} if the value is null * @throws com.clickhouse.client.api.ClientException if the column is not an array type */ - Object[] getObjectArray(int index); - - /** - * Returns reference to an array value of corresponding column. This method works for - * any multidimensional array values. However, it requires type cast so check column type. - * This method doesn't do a conversion between types. - * - * @param index column index - * @return Object - array value for the column. - * @throws com.clickhouse.client.api.ClientException if the column is not an array type - */ - Object getArray(int index); + T getObjectArray(int index); Object[] getTuple(int index); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 09ef8fe66..23b05c983 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -567,11 +567,6 @@ public Object[] getObjectArray(String colName) { return getObjectArray(schema.nameToColumnIndex(colName)); } - @Override - public Object getArray(String colName) { - return getArray(schema.nameToColumnIndex(colName)); - } - @Override public boolean hasValue(int colIndex) { return currentRecord[colIndex - 1] != null; @@ -828,6 +823,9 @@ public String[] getStringArray(int index) { BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; if (array.itemType == String.class) { return (String[]) array.getArray(); + } else if (array.itemType == BinaryStreamReader.EnumValue.class) { + BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); + return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); } else { throw new ClientException("Not an array of strings"); } @@ -849,20 +847,6 @@ public Object[] getObjectArray(int index) { throw new ClientException("Column is not of array type"); } - @Override - public Object getArray(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).getArray(); - } else if (value instanceof List) { - return ((List) value).toArray(new Object[0]); - } - throw new ClientException("Column is not of array type"); - } - @Override public Object[] getTuple(int index) { return readValue(index); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java index 14023e2b4..2f43eb92b 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java @@ -767,6 +767,8 @@ public Object[] toObjectArray() { } return result; } + + } public static class EnumValue extends Number { @@ -800,6 +802,10 @@ public double doubleValue() { return value; } + public String getName() { + return name; + } + @Override public String toString() { return name; diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java index b7e097080..3235e2ec2 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/RowBinaryTest.java @@ -79,7 +79,8 @@ void testGetObjectArray1D() { "enum_arr Array(Enum8('abc' = 1, 'cde' = 2)), " + "dt_arr Array(DateTime('UTC')), " + "fstr_arr Array(FixedString(4)), " + - "str_arr Array(String)" + + "str_arr Array(String), " + + "int_arr Array(Int32)" + ") ENGINE = MergeTree() ORDER BY tuple()"); client.execute("INSERT INTO " + table + " VALUES (" + @@ -87,7 +88,8 @@ void testGetObjectArray1D() { "['abc', 'cde'], " + "['2030-10-09 08:07:06', '2031-10-09 08:07:06'], " + "['abcd', 'efgh'], " + - "['hello', 'world'])"); + "['hello', 'world'], " + + "[100, 200, 65536])"); QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.RowBinary); TableSchema schema = client.getTableSchema(table); @@ -111,7 +113,7 @@ void testGetObjectArray1D() { Assert.assertEquals(enumArr[0].toString(), "abc"); Assert.assertEquals(enumArr[1].toString(), "cde"); - // Array(Enum8) -> String[] via getStringArray (sugar) + // Array(Enum8) -> String[] via getStringArray String[] enumStrings = reader.getStringArray("enum_arr"); Assert.assertEquals(enumStrings, new String[]{"abc", "cde"}); @@ -145,6 +147,10 @@ void testGetObjectArray1D() { String[] fstrStrings = reader.getStringArray("fstr_arr"); Assert.assertEquals(fstrStrings, new String[]{"abcd", "efgh"}); + // Array(Int32) + Object[] intArrObj = reader.getObjectArray("int_arr"); + Assert.assertEquals(intArrObj, new Integer[]{100, 200, 65536}); + Assert.assertNull(reader.next(), "Expected only one row"); } } catch (Exception e) { From 9b8cf35aa4351d41c2e43f23c8d04691451019b9 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Feb 2026 12:21:25 -0800 Subject: [PATCH 6/8] Udated GenericRecord javadoc. Fixed getStringArray() to return only arrays of strings --- .../internal/MapBackedRecord.java | 20 ++++++++----------- .../client/api/query/GenericRecord.java | 9 ++++++++- .../client/datatypes/DataTypeTests.java | 8 -------- 3 files changed, 16 insertions(+), 21 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index 1d6c5f39b..07e65de39 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -25,6 +25,7 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.temporal.TemporalAmount; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -295,19 +296,14 @@ public String[] getStringArray(String colName) { } if (value instanceof BinaryStreamReader.ArrayValue) { BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - int length = array.length; - String[] values = new String[length]; - if (array.itemType.equals(String.class)) { - for (int i = 0; i < length; i++) { - values[i] = (String) array.get(i); - } + if (array.itemType == String.class) { + return (String[]) array.getArray(); + } else if (array.itemType == BinaryStreamReader.EnumValue.class) { + BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); + return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); } else { - for (int i = 0; i < length; i++) { - Object item = array.get(i); - values[i] = item == null ? null : item.toString(); - } + throw new ClientException("Not an array of strings"); } - return values; } throw new ClientException("Column is not of array type"); } @@ -482,7 +478,7 @@ public short[] getShortArray(int index) { @Override public String[] getStringArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getStringArray(schema.columnIndexToName(index)); } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java index 5d13844ac..40ae7b87f 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java @@ -248,11 +248,18 @@ public interface GenericRecord { short[] getShortArray(String colName); + /** + * Returns string array for columns {@code Array(String)}. + * This method doesn't make a conversion of other types to string. + * + * @param colName - column name + * @return String[] + */ String[] getStringArray(String colName); /** * Reads column with name `colName` as an array of objects. Works for any array element type - * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), FixedString, etc. + * including non-primitive types like DateTime, Enum, UInt64 (BigInteger), etc. * For nested arrays, inner ArrayValue elements are recursively converted to Object[]. * * @param colName - column name diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java index ede5fc685..6c0071245 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -1332,10 +1332,6 @@ public void testGetObjectArrayMethods() throws Exception { Assert.assertEquals(uint64Arr[1], java.math.BigInteger.valueOf(200)); Assert.assertEquals(uint64Arr[2], new java.math.BigInteger("18000044073709551615")); - // Array(UInt64) -> getStringArray converts via toString() - String[] uint64Strings = row1.getStringArray("uint64_arr"); - Assert.assertEquals(uint64Strings, new String[]{"100", "200", "18000044073709551615"}); - // Array(Enum8) -> getObjectArray returns EnumValue[] Object[] enumArr = row1.getObjectArray("enum_arr"); Assert.assertNotNull(enumArr); @@ -1344,10 +1340,6 @@ public void testGetObjectArrayMethods() throws Exception { Assert.assertEquals(enumArr[0].toString(), "abc"); Assert.assertEquals(enumArr[1].toString(), "cde"); - // Array(Enum8) -> getStringArray returns enum names - String[] enumStrings = row1.getStringArray("enum_arr"); - Assert.assertEquals(enumStrings, new String[]{"abc", "cde"}); - // Array(DateTime) -> getObjectArray returns ZonedDateTime[] Object[] dtArr = row1.getObjectArray("dt_arr"); Assert.assertNotNull(dtArr); From bbfd280ac791b6ea37566fd5bc1aef6b39ae93c5 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Feb 2026 13:33:06 -0800 Subject: [PATCH 7/8] cleanup and added more tests --- .../ClickHouseBinaryFormatReader.java | 5 +- .../internal/AbstractBinaryFormatReader.java | 2304 ++++++++--------- .../internal/BinaryReaderBackedRecord.java | 898 +++---- .../internal/MapBackedRecord.java | 1372 +++++----- .../internal/BaseReaderTests.java | 122 +- .../BinaryReaderBackedRecordTest.java | 185 ++ .../client/datatypes/DataTypeTests.java | 57 + 7 files changed, 2652 insertions(+), 2291 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecordTest.java diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index 19c8b08e9..51e1b1df0 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -307,7 +307,7 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { * @param colName - column name * @return array of objects, or {@code null} if the value is null */ - T getObjectArray(String colName); + Object[] getObjectArray(String colName); /** * Reads column with name `colName` as a string. @@ -572,12 +572,11 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { * Nested arrays are recursively converted to {@code Object[]}. * Note: result is not cached so avoid repetitive calls on same column. * - * @param - type of array like {@code Object[], Integer[][]} * @param index - column index (1-based) * @return array of objects, or {@code null} if the value is null * @throws com.clickhouse.client.api.ClientException if the column is not an array type */ - T getObjectArray(int index); + Object[] getObjectArray(int index); Object[] getTuple(int index); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 23b05c983..c5c687edb 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -1,1152 +1,1152 @@ -package com.clickhouse.client.api.data_formats.internal; - -import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; -import com.clickhouse.client.api.internal.DataTypeConverter; -import com.clickhouse.client.api.internal.MapUtils; -import com.clickhouse.client.api.internal.ServerSettings; -import com.clickhouse.client.api.metadata.NoSuchColumnException; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.NullValueException; -import com.clickhouse.client.api.query.QuerySettings; -import com.clickhouse.client.api.serde.POJOFieldDeserializer; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.ClickHouseDataType; -import com.clickhouse.data.value.ClickHouseBitmap; -import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoPointValue; -import com.clickhouse.data.value.ClickHouseGeoPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoRingValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.lang.ref.WeakReference; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.net.InetAddress; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.time.temporal.TemporalAmount; -import java.util.AbstractMap; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TimeZone; -import java.util.UUID; -import java.util.function.Function; -import java.util.stream.Collectors; - -public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader { - - public static final Map> NO_TYPE_HINT_MAPPING = Collections.emptyMap(); - - private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class); - - protected InputStream input; - - protected BinaryStreamReader binaryStreamReader; - - protected DataTypeConverter dataTypeConverter; - - private TableSchema schema; - private ClickHouseColumn[] columns; - private Map[] convertions; - private boolean hasNext = true; - private boolean initialState = true; // reader is in initial state, no records have been read yet - - protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map> defaultTypeHintMap) { - this.input = inputStream; - Map settings = querySettings == null ? Collections.emptyMap() : querySettings.getAllSettings(); - Boolean useServerTimeZone = (Boolean) settings.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey()); - TimeZone timeZone = (useServerTimeZone == Boolean.TRUE && querySettings != null) ? - querySettings.getServerTimeZone() : - (TimeZone) settings.get(ClientConfigProperties.USE_TIMEZONE.getKey()); - if (timeZone == null) { - throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")"); - } - boolean jsonAsString = MapUtils.getFlag(settings, - ClientConfigProperties.serverSetting(ServerSettings.OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING), false); - this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString, - defaultTypeHintMap); - if (schema != null) { - setSchema(schema); - } - this.dataTypeConverter = DataTypeConverter.INSTANCE; // singleton while no need to customize conversion - } - - protected Object[] currentRecord; - protected Object[] nextRecord; - - protected boolean nextRecordEmpty = true; - - /** - * Reads next record into POJO object using set of serializers. - * There should be a serializer for each column in the record, otherwise it will silently skip a field - * It is done in such a way because it is not the reader concern. Calling code should validate this. - * - * Note: internal API - * @param deserializers - * @param obj - * @return - * @throws IOException - */ - public boolean readToPOJO(Map deserializers, Object obj ) throws IOException { - if (columns == null || columns.length == 0) { - return false; - } - - boolean firstColumn = true; - - for (ClickHouseColumn column : columns) { - try { - POJOFieldDeserializer deserializer = deserializers.get(column.getColumnName()); - if (deserializer != null) { - deserializer.setValue(obj, binaryStreamReader, column); - } else { - binaryStreamReader.skipValue(column); - } - firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { - endReached(); - return false; - } - throw e; - } catch (Exception e) { - throw new ClientException("Failed to set value of '" + column.getColumnName(), e); - } - } - return true; - } - - /** - * It is still internal method and should be used with care. - * Usually this method is called to read next record into internal object and affects hasNext() method. - * So after calling this one: - * - hasNext(), next() and get methods cannot be called - * - stream should be read with readRecord() method fully - * - * @param record - * @return - * @throws IOException - */ - public boolean readRecord(Map record) throws IOException { - if (columns == null || columns.length == 0) { - return false; - } - - boolean firstColumn = true; - for (ClickHouseColumn column : columns) { - try { - Object val = binaryStreamReader.readValue(column); - if (val != null) { - record.put(column.getColumnName(), val); - } else { - record.remove(column.getColumnName()); - } - firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { - endReached(); - return false; - } - throw e; - } - } - return true; - } - - protected boolean readRecord(Object[] record) throws IOException { - if (columns == null || columns.length == 0) { - return false; - } - - boolean firstColumn = true; - for (int i = 0; i < columns.length; i++) { - try { - Object val = binaryStreamReader.readValue(columns[i]); - if (val != null) { - record[i] = val; - } else { - record[i] = null; - } - firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { - endReached(); - return false; - } - throw e; - } - } - return true; - } - - @SuppressWarnings("unchecked") - @Override - public T readValue(int colIndex) { - if (colIndex < 1 || colIndex > getSchema().getColumns().size()) { - throw new ClientException("Column index out of bounds: " + colIndex); - } - return (T) currentRecord[colIndex - 1]; - } - - @SuppressWarnings("unchecked") - @Override - public T readValue(String colName) { - return (T) currentRecord[getSchema().nameToIndex(colName)]; - } - - @Override - public boolean hasNext() { - if (initialState) { - readNextRecord(); - } - - return hasNext; - } - - - protected void readNextRecord() { - initialState = false; - try { - nextRecordEmpty = true; - if (!readRecord(nextRecord)) { - endReached(); - } else { - nextRecordEmpty = false; - } - } catch (IOException e) { - endReached(); - throw new ClientException("Failed to read next row", e); - } - } - - @Override - public Map next() { - if (!hasNext) { - return null; - } - - if (!nextRecordEmpty) { - Object[] tmp = currentRecord; - currentRecord = nextRecord; - nextRecord = tmp; - readNextRecord(); - return new RecordWrapper(currentRecord, schema); - } else { - try { - if (readRecord(currentRecord)) { - readNextRecord(); - return new RecordWrapper(currentRecord, schema); - } else { - currentRecord = null; - return null; - } - } catch (IOException e) { - endReached(); - throw new ClientException("Failed to read row", e); - } - } - } - - protected void endReached() { - initialState = false; - hasNext = false; - } - - protected void setSchema(TableSchema schema) { - this.schema = schema; - this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY); - this.convertions = new Map[columns.length]; - - this.currentRecord = new Object[columns.length]; - this.nextRecord = new Object[columns.length]; - - for (int i = 0; i < columns.length; i++) { - ClickHouseColumn column = columns[i]; - ClickHouseDataType columnDataType = column.getDataType(); - if (columnDataType.equals(ClickHouseDataType.SimpleAggregateFunction)){ - columnDataType = column.getNestedColumns().get(0).getDataType(); - } - switch (columnDataType) { - case Int8: - case Int16: - case UInt8: - case Int32: - case UInt16: - case Int64: - case UInt32: - case Int128: - case UInt64: - case Int256: - case UInt128: - case UInt256: - case Float32: - case Float64: - case Decimal: - case Decimal32: - case Decimal64: - case Decimal128: - case Decimal256: - case Bool: - case String: - case Enum8: - case Enum16: - case Variant: - case Dynamic: - this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; - break; - default: - this.convertions[i] = Collections.emptyMap(); - } - } - } - - public Map[] getConvertions() { - return convertions; - } - - @Override - public TableSchema getSchema() { - return schema; - } - - @Override - public String getString(String colName) { - return getString(schema.nameToColumnIndex(colName)); - } - - @Override - public String getString(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - Object value; - switch (column.getValueDataType()) { - case Date: - case Date32: - value = getLocalDate(index); - break; - case Time: - case Time64: - value = getLocalTime(index); - break; - case DateTime: - case DateTime32: - case DateTime64: - value = getLocalDateTime(index); - break; - default: - value = readValue(index); - } - - return dataTypeConverter.convertToString(value, column); - } - - @SuppressWarnings("unchecked") - private T readNumberValue(int index, NumberConverter.NumberType targetType) { - int colIndex = index - 1; - Function converter = (Function) convertions[colIndex].get(targetType); - if (converter != null) { - Object value = readValue(index); - if (value == null) { - if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { - return null; - } - throw new NullValueException("Column at index " + index + " has null value and it cannot be cast to " + - targetType.getTypeName()); - } - return (T) converter.apply(value); - } else { - throw new ClientException("Column at index " + index + " " + columns[colIndex].getDataType().name() + - " cannot be converted to " + targetType.getTypeName()); - } - } - - @Override - public byte getByte(String colName) { - return getByte(schema.nameToColumnIndex(colName)); - } - - @Override - public short getShort(String colName) { - return getShort(schema.nameToColumnIndex(colName)); - } - - @Override - public int getInteger(String colName) { - return getInteger(schema.nameToColumnIndex(colName)); - } - - @Override - public long getLong(String colName) { - return getLong(schema.nameToColumnIndex(colName)); - } - - @Override - public float getFloat(String colName) { - return getFloat(schema.nameToColumnIndex(colName)); - } - - @Override - public double getDouble(String colName) { - return getDouble(schema.nameToColumnIndex(colName)); - } - - @Override - public boolean getBoolean(String colName) { - return getBoolean(schema.nameToColumnIndex(colName)); - } - - @Override - public BigInteger getBigInteger(String colName) { - return getBigInteger(schema.nameToColumnIndex(colName)); - } - - @Override - public BigDecimal getBigDecimal(String colName) { - return getBigDecimal(schema.nameToColumnIndex(colName)); - } - - @Override - public Instant getInstant(String colName) { - return getInstant(getSchema().nameToColumnIndex(colName)); - } - - @Override - public ZonedDateTime getZonedDateTime(String colName) { - return getZonedDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public Duration getDuration(String colName) { - return getDuration(schema.nameToColumnIndex(colName)); - } - - @Override - public TemporalAmount getTemporalAmount(String colName) { - return getTemporalAmount(schema.nameToColumnIndex(colName)); - } - - @Override - public Inet4Address getInet4Address(String colName) { - return getInet4Address(schema.nameToColumnIndex(colName)); - } - - @Override - public Inet6Address getInet6Address(String colName) { - return getInet6Address(schema.nameToColumnIndex(colName)); - } - - @Override - public UUID getUUID(String colName) { - return getUUID(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(String colName) { - return getGeoPoint(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(String colName) { - return getGeoRing(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { - return getGeoPolygon(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { - return getGeoMultiPolygon(schema.nameToColumnIndex(colName)); - } - - - @Override - public List getList(String colName) { - return getList(schema.nameToColumnIndex(colName)); - } - - - @SuppressWarnings("unchecked") - private T getPrimitiveArray(int index, Class componentType) { - try { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - if (array.itemType.isPrimitive()) { - return (T) array.array; - } else { - throw new ClientException("Array is not of primitive type"); - } - } else if (value instanceof List) { - List list = (List) value; - Object array = Array.newInstance(componentType, list.size()); - for (int i = 0; i < list.size(); i++) { - Array.set(array, i, list.get(i)); - } - return (T)array; - } else if (componentType == byte.class) { - if (value instanceof String) { - return (T) ((String) value).getBytes(StandardCharsets.UTF_8); - } else if (value instanceof InetAddress) { - return (T) ((InetAddress) value).getAddress(); - } - } - throw new ClientException("Column is not of array type"); - } catch (ClassCastException e) { - throw new ClientException("Column is not of array type", e); - } - } - - @Override - public byte[] getByteArray(String colName) { - return getByteArray(schema.nameToColumnIndex(colName)); - } - - @Override - public int[] getIntArray(String colName) { - return getIntArray(schema.nameToColumnIndex(colName)); - } - - @Override - public long[] getLongArray(String colName) { - return getLongArray(schema.nameToColumnIndex(colName)); - } - - @Override - public float[] getFloatArray(String colName) { - return getFloatArray(schema.nameToColumnIndex(colName)); - } - - @Override - public double[] getDoubleArray(String colName) { - return getDoubleArray(schema.nameToColumnIndex(colName)); - } - - @Override - public boolean[] getBooleanArray(String colName) { - return getBooleanArray(schema.nameToColumnIndex(colName)); - } - - @Override - public short[] getShortArray(String colName) { - return getShortArray(schema.nameToColumnIndex(colName)); - } - - @Override - public String[] getStringArray(String colName) { - return getStringArray(schema.nameToColumnIndex(colName)); - } - - @Override - public Object[] getObjectArray(String colName) { - return getObjectArray(schema.nameToColumnIndex(colName)); - } - - @Override - public boolean hasValue(int colIndex) { - return currentRecord[colIndex - 1] != null; - } - - @Override - public boolean hasValue(String colName) { - return hasValue(schema.nameToColumnIndex(colName)); - } - - @Override - public byte getByte(int index) { - return readNumberValue(index, NumberConverter.NumberType.Byte); - } - - @Override - public short getShort(int index) { - return readNumberValue(index, NumberConverter.NumberType.Short); - } - - @Override - public int getInteger(int index) { - return readNumberValue(index, NumberConverter.NumberType.Int); - } - - @Override - public long getLong(int index) { - return readNumberValue(index, NumberConverter.NumberType.Long); - } - - @Override - public float getFloat(int index) { - return readNumberValue(index, NumberConverter.NumberType.Float); - } - - @Override - public double getDouble(int index) { - return readNumberValue(index, NumberConverter.NumberType.Double); - } - - @Override - public boolean getBoolean(int index) { - return readNumberValue(index, NumberConverter.NumberType.Boolean); - } - - @Override - public BigInteger getBigInteger(int index) { - return readNumberValue(index, NumberConverter.NumberType.BigInteger); - } - - @Override - public BigDecimal getBigDecimal(int index) { - return readNumberValue(index, NumberConverter.NumberType.BigDecimal); - } - - @Override - public Instant getInstant(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch (column.getValueDataType()) { - case Date: - case Date32: - LocalDate date = getLocalDate(index); - return date == null ? null : date.atStartOfDay(ZoneId.of("UTC")).toInstant(); - case Time: - case Time64: - LocalDateTime dt = getLocalDateTime(index); - return dt == null ? null : dt.toInstant(ZoneOffset.UTC); - case DateTime: - case DateTime64: - case DateTime32: - ZonedDateTime zdt = readValue(index); - return zdt.toInstant(); - case Dynamic: - case Variant: - Object value = readValue(index); - Instant instant = objectToInstant(value); - if (value == null || instant != null) { - return instant; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); - } - - static Instant objectToInstant(Object value) { - if (value instanceof LocalDateTime) { - LocalDateTime dateTime = (LocalDateTime) value; - return Instant.from(dateTime.atZone(ZoneId.of("UTC"))); - } else if (value instanceof ZonedDateTime) { - ZonedDateTime dateTime = (ZonedDateTime) value; - return dateTime.toInstant(); - } - return null; - } - - @Override - public ZonedDateTime getZonedDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch (column.getValueDataType()) { - case DateTime: - case DateTime64: - case DateTime32: - return readValue(index); - case Dynamic: - case Variant: - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof ZonedDateTime) { - return (ZonedDateTime) value; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); - } - - @Override - public Duration getDuration(int index) { - TemporalAmount temporalAmount = getTemporalAmount(index); - return temporalAmount == null ? null : Duration.from(temporalAmount); - } - - @Override - public TemporalAmount getTemporalAmount(int index) { - return readValue(index); - } - - @Override - public Inet4Address getInet4Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); - } - - @Override - public Inet6Address getInet6Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); - } - - @Override - public UUID getUUID(int index) { - return readValue(index); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); - } - - @Override - public List getList(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).asList(); - } else if (value instanceof List) { - return (List) value; - } else { - throw new ClientException("Column is not of array type"); - } - } - - @Override - public byte[] getByteArray(int index) { - try { - return getPrimitiveArray(index, byte.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public int[] getIntArray(int index) { - try { - return getPrimitiveArray(index, int.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public long[] getLongArray(int index) { - try { - return getPrimitiveArray(index, long.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public float[] getFloatArray(int index) { - try { - return getPrimitiveArray(index, float.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public double[] getDoubleArray(int index) { - try { - return getPrimitiveArray(index, double.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public boolean[] getBooleanArray(int index) { - try { - return getPrimitiveArray(index, boolean.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public short[] getShortArray(int index) { - try { - return getPrimitiveArray(index, short.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public String[] getStringArray(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - if (array.itemType == String.class) { - return (String[]) array.getArray(); - } else if (array.itemType == BinaryStreamReader.EnumValue.class) { - BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); - return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); - } else { - throw new ClientException("Not an array of strings"); - } - } - throw new ClientException("Column is not of array type"); - } - - @Override - public Object[] getObjectArray(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); - } else if (value instanceof List) { - return ((List) value).toArray(new Object[0]); - } - throw new ClientException("Column is not of array type"); - } - - @Override - public Object[] getTuple(int index) { - return readValue(index); - } - - @Override - public Object[] getTuple(String colName) { - return getTuple(schema.nameToColumnIndex(colName)); - } - - @Override - public byte getEnum8(String colName) { - return getEnum8(schema.nameToColumnIndex(colName)); - } - - @Override - public byte getEnum8(int index) { - BinaryStreamReader.EnumValue enumValue = readValue(index); - if (enumValue == null) { - throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum8 numeric value"); - } - return enumValue.byteValue(); - } - - @Override - public short getEnum16(String colName) { - return getEnum16(schema.nameToColumnIndex(colName)); - } - - @Override - public short getEnum16(int index) { - BinaryStreamReader.EnumValue enumValue = readValue(index); - if (enumValue == null) { - throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum16 numeric value"); - } - return enumValue.shortValue(); - } - - @Override - public LocalDate getLocalDate(String colName) { - return getLocalDate(schema.nameToColumnIndex(colName)); - } - - @Override - public LocalDate getLocalDate(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case Date: - case Date32: - return readValue(index); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toLocalDate(); - case Dynamic: - case Variant: - Object value = readValue(index); - LocalDate localDate = objectToLocalDate(value); - if (value == null || localDate != null) { - return localDate; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); - } - - static LocalDate objectToLocalDate(Object value) { - if (value instanceof LocalDate) { - return (LocalDate) value; - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime)value).toLocalDate(); - } else if (value instanceof LocalDateTime) { - return ((LocalDateTime)value).toLocalDate(); - } - return null; - } - - @Override - public LocalTime getLocalTime(String colName) { - return getLocalTime(schema.nameToColumnIndex(colName)); - } - - @Override - public LocalTime getLocalTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case Time: - case Time64: - LocalDateTime dt = readValue(index); - return dt == null ? null : dt.toLocalTime(); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toLocalTime(); - case Dynamic: - case Variant: - Object value = readValue(index); - LocalTime localTime = objectToLocalTime(value); - if (value == null || localTime != null) { - return localTime; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); - } - - static LocalTime objectToLocalTime(Object value) { - if (value instanceof LocalDateTime) { - return ((LocalDateTime)value).toLocalTime(); - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime)value).toLocalTime(); - } - return null; - } - - @Override - public LocalDateTime getLocalDateTime(String colName) { - return getLocalDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public LocalDateTime getLocalDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case Time: - case Time64: - return readValue(index); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toLocalDateTime(); - case Dynamic: - case Variant: - Object value = readValue(index); - LocalDateTime ldt = objectToLocalDateTime(value); - if (value == null || ldt != null) { - return ldt; - } - break; - - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); - } - - static LocalDateTime objectToLocalDateTime(Object value) { - if (value instanceof LocalDateTime) { - return (LocalDateTime) value; - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime)value).toLocalDateTime(); - } - - return null; - } - - @Override - public OffsetDateTime getOffsetDateTime(String colName) { - return getOffsetDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public OffsetDateTime getOffsetDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toOffsetDateTime(); - case Dynamic: - case Variant: - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toOffsetDateTime(); - } - - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDateTime"); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(String colName) { - return getClickHouseBitmap(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(int index) { - return readValue(index); - } - - @Override - public void close() throws Exception { - input.close(); - } - - private static class RecordWrapper implements Map { - - private final WeakReference recordRef; - - private final WeakReference schemaRef; - - int size; - public RecordWrapper(Object[] record, TableSchema schema) { - this.recordRef = new WeakReference<>(record); - this.schemaRef = new WeakReference<>(schema); - this.size = record.length; - } - - @Override - public int size() { - return size; - } - - @Override - public boolean isEmpty() { - return size == 0; - } - - @Override - @SuppressWarnings("ConstantConditions") - public boolean containsKey(Object key) { - if (key instanceof String) { - return recordRef.get()[schemaRef.get().nameToIndex((String)key)] != null; - } - return false; - } - - @Override - public boolean containsValue(Object value) { - for (Object obj : recordRef.get()) { - if (obj == value) { - return true; - } - } - return false; - } - - @Override - @SuppressWarnings("ConstantConditions") - public Object get(Object key) { - if (key instanceof String) { - try { - int index = schemaRef.get().nameToIndex((String) key); - if (index < size) { - return recordRef.get()[index]; - } - } catch (NoSuchColumnException e) { - return null; - } - } - - return null; - } - - @Override - public Object put(String key, Object value) { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - public Object remove(Object key) { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - public void putAll(Map m) { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - public void clear() { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - @SuppressWarnings("ConstantConditions") - public Set keySet() { - // TODO: create a view in Schema - return schemaRef.get().getColumns().stream().map(ClickHouseColumn::getColumnName).collect(Collectors.toSet()); - } - - @Override - @SuppressWarnings("ConstantConditions") - public Collection values() { - return Arrays.asList(recordRef.get()); - } - - @Override - @SuppressWarnings("ConstantConditions") - public Set> entrySet() { - int i = 0; - Set> entrySet = new HashSet<>(); - for (ClickHouseColumn column : schemaRef.get().getColumns()) { - entrySet.add( new AbstractMap.SimpleImmutableEntry(column.getColumnName(), recordRef.get()[i++])); - } - return entrySet; - } - } -} +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.internal.DataTypeConverter; +import com.clickhouse.client.api.internal.MapUtils; +import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.metadata.NoSuchColumnException; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.NullValueException; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.client.api.serde.POJOFieldDeserializer; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.lang.ref.WeakReference; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.net.InetAddress; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TimeZone; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; + +public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader { + + public static final Map> NO_TYPE_HINT_MAPPING = Collections.emptyMap(); + + private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class); + + protected InputStream input; + + protected BinaryStreamReader binaryStreamReader; + + protected DataTypeConverter dataTypeConverter; + + private TableSchema schema; + private ClickHouseColumn[] columns; + private Map[] convertions; + private boolean hasNext = true; + private boolean initialState = true; // reader is in initial state, no records have been read yet + + protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map> defaultTypeHintMap) { + this.input = inputStream; + Map settings = querySettings == null ? Collections.emptyMap() : querySettings.getAllSettings(); + Boolean useServerTimeZone = (Boolean) settings.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey()); + TimeZone timeZone = (useServerTimeZone == Boolean.TRUE && querySettings != null) ? + querySettings.getServerTimeZone() : + (TimeZone) settings.get(ClientConfigProperties.USE_TIMEZONE.getKey()); + if (timeZone == null) { + throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")"); + } + boolean jsonAsString = MapUtils.getFlag(settings, + ClientConfigProperties.serverSetting(ServerSettings.OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING), false); + this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString, + defaultTypeHintMap); + if (schema != null) { + setSchema(schema); + } + this.dataTypeConverter = DataTypeConverter.INSTANCE; // singleton while no need to customize conversion + } + + protected Object[] currentRecord; + protected Object[] nextRecord; + + protected boolean nextRecordEmpty = true; + + /** + * Reads next record into POJO object using set of serializers. + * There should be a serializer for each column in the record, otherwise it will silently skip a field + * It is done in such a way because it is not the reader concern. Calling code should validate this. + * + * Note: internal API + * @param deserializers + * @param obj + * @return + * @throws IOException + */ + public boolean readToPOJO(Map deserializers, Object obj ) throws IOException { + if (columns == null || columns.length == 0) { + return false; + } + + boolean firstColumn = true; + + for (ClickHouseColumn column : columns) { + try { + POJOFieldDeserializer deserializer = deserializers.get(column.getColumnName()); + if (deserializer != null) { + deserializer.setValue(obj, binaryStreamReader, column); + } else { + binaryStreamReader.skipValue(column); + } + firstColumn = false; + } catch (EOFException e) { + if (firstColumn) { + endReached(); + return false; + } + throw e; + } catch (Exception e) { + throw new ClientException("Failed to set value of '" + column.getColumnName(), e); + } + } + return true; + } + + /** + * It is still internal method and should be used with care. + * Usually this method is called to read next record into internal object and affects hasNext() method. + * So after calling this one: + * - hasNext(), next() and get methods cannot be called + * - stream should be read with readRecord() method fully + * + * @param record + * @return + * @throws IOException + */ + public boolean readRecord(Map record) throws IOException { + if (columns == null || columns.length == 0) { + return false; + } + + boolean firstColumn = true; + for (ClickHouseColumn column : columns) { + try { + Object val = binaryStreamReader.readValue(column); + if (val != null) { + record.put(column.getColumnName(), val); + } else { + record.remove(column.getColumnName()); + } + firstColumn = false; + } catch (EOFException e) { + if (firstColumn) { + endReached(); + return false; + } + throw e; + } + } + return true; + } + + protected boolean readRecord(Object[] record) throws IOException { + if (columns == null || columns.length == 0) { + return false; + } + + boolean firstColumn = true; + for (int i = 0; i < columns.length; i++) { + try { + Object val = binaryStreamReader.readValue(columns[i]); + if (val != null) { + record[i] = val; + } else { + record[i] = null; + } + firstColumn = false; + } catch (EOFException e) { + if (firstColumn) { + endReached(); + return false; + } + throw e; + } + } + return true; + } + + @SuppressWarnings("unchecked") + @Override + public T readValue(int colIndex) { + if (colIndex < 1 || colIndex > getSchema().getColumns().size()) { + throw new ClientException("Column index out of bounds: " + colIndex); + } + return (T) currentRecord[colIndex - 1]; + } + + @SuppressWarnings("unchecked") + @Override + public T readValue(String colName) { + return (T) currentRecord[getSchema().nameToIndex(colName)]; + } + + @Override + public boolean hasNext() { + if (initialState) { + readNextRecord(); + } + + return hasNext; + } + + + protected void readNextRecord() { + initialState = false; + try { + nextRecordEmpty = true; + if (!readRecord(nextRecord)) { + endReached(); + } else { + nextRecordEmpty = false; + } + } catch (IOException e) { + endReached(); + throw new ClientException("Failed to read next row", e); + } + } + + @Override + public Map next() { + if (!hasNext) { + return null; + } + + if (!nextRecordEmpty) { + Object[] tmp = currentRecord; + currentRecord = nextRecord; + nextRecord = tmp; + readNextRecord(); + return new RecordWrapper(currentRecord, schema); + } else { + try { + if (readRecord(currentRecord)) { + readNextRecord(); + return new RecordWrapper(currentRecord, schema); + } else { + currentRecord = null; + return null; + } + } catch (IOException e) { + endReached(); + throw new ClientException("Failed to read row", e); + } + } + } + + protected void endReached() { + initialState = false; + hasNext = false; + } + + protected void setSchema(TableSchema schema) { + this.schema = schema; + this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY); + this.convertions = new Map[columns.length]; + + this.currentRecord = new Object[columns.length]; + this.nextRecord = new Object[columns.length]; + + for (int i = 0; i < columns.length; i++) { + ClickHouseColumn column = columns[i]; + ClickHouseDataType columnDataType = column.getDataType(); + if (columnDataType.equals(ClickHouseDataType.SimpleAggregateFunction)){ + columnDataType = column.getNestedColumns().get(0).getDataType(); + } + switch (columnDataType) { + case Int8: + case Int16: + case UInt8: + case Int32: + case UInt16: + case Int64: + case UInt32: + case Int128: + case UInt64: + case Int256: + case UInt128: + case UInt256: + case Float32: + case Float64: + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + case Bool: + case String: + case Enum8: + case Enum16: + case Variant: + case Dynamic: + this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; + break; + default: + this.convertions[i] = Collections.emptyMap(); + } + } + } + + public Map[] getConvertions() { + return convertions; + } + + @Override + public TableSchema getSchema() { + return schema; + } + + @Override + public String getString(String colName) { + return getString(schema.nameToColumnIndex(colName)); + } + + @Override + public String getString(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + Object value; + switch (column.getValueDataType()) { + case Date: + case Date32: + value = getLocalDate(index); + break; + case Time: + case Time64: + value = getLocalTime(index); + break; + case DateTime: + case DateTime32: + case DateTime64: + value = getLocalDateTime(index); + break; + default: + value = readValue(index); + } + + return dataTypeConverter.convertToString(value, column); + } + + @SuppressWarnings("unchecked") + private T readNumberValue(int index, NumberConverter.NumberType targetType) { + int colIndex = index - 1; + Function converter = (Function) convertions[colIndex].get(targetType); + if (converter != null) { + Object value = readValue(index); + if (value == null) { + if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { + return null; + } + throw new NullValueException("Column at index " + index + " has null value and it cannot be cast to " + + targetType.getTypeName()); + } + return (T) converter.apply(value); + } else { + throw new ClientException("Column at index " + index + " " + columns[colIndex].getDataType().name() + + " cannot be converted to " + targetType.getTypeName()); + } + } + + @Override + public byte getByte(String colName) { + return getByte(schema.nameToColumnIndex(colName)); + } + + @Override + public short getShort(String colName) { + return getShort(schema.nameToColumnIndex(colName)); + } + + @Override + public int getInteger(String colName) { + return getInteger(schema.nameToColumnIndex(colName)); + } + + @Override + public long getLong(String colName) { + return getLong(schema.nameToColumnIndex(colName)); + } + + @Override + public float getFloat(String colName) { + return getFloat(schema.nameToColumnIndex(colName)); + } + + @Override + public double getDouble(String colName) { + return getDouble(schema.nameToColumnIndex(colName)); + } + + @Override + public boolean getBoolean(String colName) { + return getBoolean(schema.nameToColumnIndex(colName)); + } + + @Override + public BigInteger getBigInteger(String colName) { + return getBigInteger(schema.nameToColumnIndex(colName)); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + return getBigDecimal(schema.nameToColumnIndex(colName)); + } + + @Override + public Instant getInstant(String colName) { + return getInstant(getSchema().nameToColumnIndex(colName)); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + return getZonedDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public Duration getDuration(String colName) { + return getDuration(schema.nameToColumnIndex(colName)); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return getTemporalAmount(schema.nameToColumnIndex(colName)); + } + + @Override + public Inet4Address getInet4Address(String colName) { + return getInet4Address(schema.nameToColumnIndex(colName)); + } + + @Override + public Inet6Address getInet6Address(String colName) { + return getInet6Address(schema.nameToColumnIndex(colName)); + } + + @Override + public UUID getUUID(String colName) { + return getUUID(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + return getGeoPoint(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + return getGeoRing(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + return getGeoPolygon(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + return getGeoMultiPolygon(schema.nameToColumnIndex(colName)); + } + + + @Override + public List getList(String colName) { + return getList(schema.nameToColumnIndex(colName)); + } + + + @SuppressWarnings("unchecked") + private T getPrimitiveArray(int index, Class componentType) { + try { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType.isPrimitive()) { + return (T) array.array; + } else { + throw new ClientException("Array is not of primitive type"); + } + } else if (value instanceof List) { + List list = (List) value; + Object array = Array.newInstance(componentType, list.size()); + for (int i = 0; i < list.size(); i++) { + Array.set(array, i, list.get(i)); + } + return (T)array; + } else if (componentType == byte.class) { + if (value instanceof String) { + return (T) ((String) value).getBytes(StandardCharsets.UTF_8); + } else if (value instanceof InetAddress) { + return (T) ((InetAddress) value).getAddress(); + } + } + throw new ClientException("Column is not of array type"); + } catch (ClassCastException e) { + throw new ClientException("Column is not of array type", e); + } + } + + @Override + public byte[] getByteArray(String colName) { + return getByteArray(schema.nameToColumnIndex(colName)); + } + + @Override + public int[] getIntArray(String colName) { + return getIntArray(schema.nameToColumnIndex(colName)); + } + + @Override + public long[] getLongArray(String colName) { + return getLongArray(schema.nameToColumnIndex(colName)); + } + + @Override + public float[] getFloatArray(String colName) { + return getFloatArray(schema.nameToColumnIndex(colName)); + } + + @Override + public double[] getDoubleArray(String colName) { + return getDoubleArray(schema.nameToColumnIndex(colName)); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return getBooleanArray(schema.nameToColumnIndex(colName)); + } + + @Override + public short[] getShortArray(String colName) { + return getShortArray(schema.nameToColumnIndex(colName)); + } + + @Override + public String[] getStringArray(String colName) { + return getStringArray(schema.nameToColumnIndex(colName)); + } + + @Override + public Object[] getObjectArray(String colName) { + return getObjectArray(schema.nameToColumnIndex(colName)); + } + + @Override + public boolean hasValue(int colIndex) { + return currentRecord[colIndex - 1] != null; + } + + @Override + public boolean hasValue(String colName) { + return hasValue(schema.nameToColumnIndex(colName)); + } + + @Override + public byte getByte(int index) { + return readNumberValue(index, NumberConverter.NumberType.Byte); + } + + @Override + public short getShort(int index) { + return readNumberValue(index, NumberConverter.NumberType.Short); + } + + @Override + public int getInteger(int index) { + return readNumberValue(index, NumberConverter.NumberType.Int); + } + + @Override + public long getLong(int index) { + return readNumberValue(index, NumberConverter.NumberType.Long); + } + + @Override + public float getFloat(int index) { + return readNumberValue(index, NumberConverter.NumberType.Float); + } + + @Override + public double getDouble(int index) { + return readNumberValue(index, NumberConverter.NumberType.Double); + } + + @Override + public boolean getBoolean(int index) { + return readNumberValue(index, NumberConverter.NumberType.Boolean); + } + + @Override + public BigInteger getBigInteger(int index) { + return readNumberValue(index, NumberConverter.NumberType.BigInteger); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return readNumberValue(index, NumberConverter.NumberType.BigDecimal); + } + + @Override + public Instant getInstant(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch (column.getValueDataType()) { + case Date: + case Date32: + LocalDate date = getLocalDate(index); + return date == null ? null : date.atStartOfDay(ZoneId.of("UTC")).toInstant(); + case Time: + case Time64: + LocalDateTime dt = getLocalDateTime(index); + return dt == null ? null : dt.toInstant(ZoneOffset.UTC); + case DateTime: + case DateTime64: + case DateTime32: + ZonedDateTime zdt = readValue(index); + return zdt.toInstant(); + case Dynamic: + case Variant: + Object value = readValue(index); + Instant instant = objectToInstant(value); + if (value == null || instant != null) { + return instant; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); + } + + static Instant objectToInstant(Object value) { + if (value instanceof LocalDateTime) { + LocalDateTime dateTime = (LocalDateTime) value; + return Instant.from(dateTime.atZone(ZoneId.of("UTC"))); + } else if (value instanceof ZonedDateTime) { + ZonedDateTime dateTime = (ZonedDateTime) value; + return dateTime.toInstant(); + } + return null; + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch (column.getValueDataType()) { + case DateTime: + case DateTime64: + case DateTime32: + return readValue(index); + case Dynamic: + case Variant: + Object value = readValue(index); + if (value == null) { + return null; + } else if (value instanceof ZonedDateTime) { + return (ZonedDateTime) value; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); + } + + @Override + public Duration getDuration(int index) { + TemporalAmount temporalAmount = getTemporalAmount(index); + return temporalAmount == null ? null : Duration.from(temporalAmount); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return readValue(index); + } + + @Override + public Inet4Address getInet4Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); + } + + @Override + public Inet6Address getInet6Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); + } + + @Override + public UUID getUUID(int index) { + return readValue(index); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); + } + + @Override + public List getList(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).asList(); + } else if (value instanceof List) { + return (List) value; + } else { + throw new ClientException("Column is not of array type"); + } + } + + @Override + public byte[] getByteArray(int index) { + try { + return getPrimitiveArray(index, byte.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public int[] getIntArray(int index) { + try { + return getPrimitiveArray(index, int.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public long[] getLongArray(int index) { + try { + return getPrimitiveArray(index, long.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public float[] getFloatArray(int index) { + try { + return getPrimitiveArray(index, float.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public double[] getDoubleArray(int index) { + try { + return getPrimitiveArray(index, double.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public boolean[] getBooleanArray(int index) { + try { + return getPrimitiveArray(index, boolean.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public short[] getShortArray(int index) { + try { + return getPrimitiveArray(index, short.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public String[] getStringArray(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType == String.class) { + return (String[]) array.getArray(); + } else if (array.itemType == BinaryStreamReader.EnumValue.class) { + BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); + return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); + } else { + throw new ClientException("Not an array of strings"); + } + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getObjectArray(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getTuple(int index) { + return readValue(index); + } + + @Override + public Object[] getTuple(String colName) { + return getTuple(schema.nameToColumnIndex(colName)); + } + + @Override + public byte getEnum8(String colName) { + return getEnum8(schema.nameToColumnIndex(colName)); + } + + @Override + public byte getEnum8(int index) { + BinaryStreamReader.EnumValue enumValue = readValue(index); + if (enumValue == null) { + throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum8 numeric value"); + } + return enumValue.byteValue(); + } + + @Override + public short getEnum16(String colName) { + return getEnum16(schema.nameToColumnIndex(colName)); + } + + @Override + public short getEnum16(int index) { + BinaryStreamReader.EnumValue enumValue = readValue(index); + if (enumValue == null) { + throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum16 numeric value"); + } + return enumValue.shortValue(); + } + + @Override + public LocalDate getLocalDate(String colName) { + return getLocalDate(schema.nameToColumnIndex(colName)); + } + + @Override + public LocalDate getLocalDate(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case Date: + case Date32: + return readValue(index); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toLocalDate(); + case Dynamic: + case Variant: + Object value = readValue(index); + LocalDate localDate = objectToLocalDate(value); + if (value == null || localDate != null) { + return localDate; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); + } + + static LocalDate objectToLocalDate(Object value) { + if (value instanceof LocalDate) { + return (LocalDate) value; + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime)value).toLocalDate(); + } else if (value instanceof LocalDateTime) { + return ((LocalDateTime)value).toLocalDate(); + } + return null; + } + + @Override + public LocalTime getLocalTime(String colName) { + return getLocalTime(schema.nameToColumnIndex(colName)); + } + + @Override + public LocalTime getLocalTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case Time: + case Time64: + LocalDateTime dt = readValue(index); + return dt == null ? null : dt.toLocalTime(); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toLocalTime(); + case Dynamic: + case Variant: + Object value = readValue(index); + LocalTime localTime = objectToLocalTime(value); + if (value == null || localTime != null) { + return localTime; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); + } + + static LocalTime objectToLocalTime(Object value) { + if (value instanceof LocalDateTime) { + return ((LocalDateTime)value).toLocalTime(); + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime)value).toLocalTime(); + } + return null; + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + return getLocalDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case Time: + case Time64: + return readValue(index); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toLocalDateTime(); + case Dynamic: + case Variant: + Object value = readValue(index); + LocalDateTime ldt = objectToLocalDateTime(value); + if (value == null || ldt != null) { + return ldt; + } + break; + + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); + } + + static LocalDateTime objectToLocalDateTime(Object value) { + if (value instanceof LocalDateTime) { + return (LocalDateTime) value; + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime)value).toLocalDateTime(); + } + + return null; + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + return getOffsetDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toOffsetDateTime(); + case Dynamic: + case Variant: + Object value = readValue(index); + if (value == null) { + return null; + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime) value).toOffsetDateTime(); + } + + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDateTime"); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + return getClickHouseBitmap(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + return readValue(index); + } + + @Override + public void close() throws Exception { + input.close(); + } + + private static class RecordWrapper implements Map { + + private final WeakReference recordRef; + + private final WeakReference schemaRef; + + int size; + public RecordWrapper(Object[] record, TableSchema schema) { + this.recordRef = new WeakReference<>(record); + this.schemaRef = new WeakReference<>(schema); + this.size = record.length; + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return size == 0; + } + + @Override + @SuppressWarnings("ConstantConditions") + public boolean containsKey(Object key) { + if (key instanceof String) { + return recordRef.get()[schemaRef.get().nameToIndex((String)key)] != null; + } + return false; + } + + @Override + public boolean containsValue(Object value) { + for (Object obj : recordRef.get()) { + if (obj == value) { + return true; + } + } + return false; + } + + @Override + @SuppressWarnings("ConstantConditions") + public Object get(Object key) { + if (key instanceof String) { + try { + int index = schemaRef.get().nameToIndex((String) key); + if (index < size) { + return recordRef.get()[index]; + } + } catch (NoSuchColumnException e) { + return null; + } + } + + return null; + } + + @Override + public Object put(String key, Object value) { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + public Object remove(Object key) { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + public void putAll(Map m) { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + public void clear() { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + @SuppressWarnings("ConstantConditions") + public Set keySet() { + // TODO: create a view in Schema + return schemaRef.get().getColumns().stream().map(ClickHouseColumn::getColumnName).collect(Collectors.toSet()); + } + + @Override + @SuppressWarnings("ConstantConditions") + public Collection values() { + return Arrays.asList(recordRef.get()); + } + + @Override + @SuppressWarnings("ConstantConditions") + public Set> entrySet() { + int i = 0; + Set> entrySet = new HashSet<>(); + for (ClickHouseColumn column : schemaRef.get().getColumns()) { + entrySet.add( new AbstractMap.SimpleImmutableEntry(column.getColumnName(), recordRef.get()[i++])); + } + return entrySet; + } + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java index 7faa8e0fe..4739ca83d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java @@ -1,449 +1,449 @@ -package com.clickhouse.client.api.data_formats.internal; - -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.value.*; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.time.*; -import java.time.temporal.TemporalAmount; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Collectors; - -public class BinaryReaderBackedRecord implements GenericRecord { - - private final ClickHouseBinaryFormatReader reader; - - public BinaryReaderBackedRecord(ClickHouseBinaryFormatReader reader) { - this.reader = reader; - } - - @Override - public String getString(String colName) { - return reader.getString(colName); - } - - @Override - public byte getByte(String colName) { - return reader.getByte(colName); - } - - @Override - public short getShort(String colName) { - return reader.getShort(colName); - } - - @Override - public int getInteger(String colName) { - return reader.getInteger(colName); - } - - @Override - public long getLong(String colName) { - return reader.getLong(colName); - } - - @Override - public float getFloat(String colName) { - return reader.getFloat(colName); - } - - @Override - public double getDouble(String colName) { - return reader.getDouble(colName); - } - - @Override - public boolean getBoolean(String colName) { - return reader.getBoolean(colName); - } - - @Override - public BigInteger getBigInteger(String colName) { - return reader.getBigInteger(colName); - } - - @Override - public BigDecimal getBigDecimal(String colName) { - return reader.getBigDecimal(colName); - } - - @Override - public Instant getInstant(String colName) { - return reader.getInstant(colName); - } - - @Override - public ZonedDateTime getZonedDateTime(String colName) { - return reader.getZonedDateTime(colName); - } - - @Override - public Duration getDuration(String colName) { - return reader.getDuration(colName); - } - - @Override - public TemporalAmount getTemporalAmount(String colName) { - return reader.getTemporalAmount(colName); - } - - @Override - public Inet4Address getInet4Address(String colName) { - return reader.getInet4Address(colName); - } - - @Override - public Inet6Address getInet6Address(String colName) { - return reader.getInet6Address(colName); - } - - @Override - public UUID getUUID(String colName) { - return reader.getUUID(colName); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(String colName) { - return reader.getGeoPoint(colName); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(String colName) { - return reader.getGeoRing(colName); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { - return reader.getGeoPolygon(colName); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { - return reader.getGeoMultiPolygon(colName); - } - - @Override - public List getList(String colName) { - return reader.getList(colName); - } - - @Override - public byte[] getByteArray(String colName) { - return reader.getByteArray(colName); - } - - @Override - public int[] getIntArray(String colName) { - return reader.getIntArray(colName); - } - - @Override - public long[] getLongArray(String colName) { - return reader.getLongArray(colName); - } - - @Override - public float[] getFloatArray(String colName) { - return reader.getFloatArray(colName); - } - - @Override - public double[] getDoubleArray(String colName) { - return reader.getDoubleArray(colName); - } - - @Override - public boolean[] getBooleanArray(String colName) { - return reader.getBooleanArray(colName); - } - - @Override - public short[] getShortArray(String colName) { - return reader.getShortArray(colName); - } - - @Override - public String[] getStringArray(String colName) { - return reader.getStringArray(colName); - } - - @Override - public Object[] getObjectArray(String colName) { - return reader.getObjectArray(colName); - } - - @Override - public String getString(int index) { - return reader.getString(index); - } - - @Override - public boolean hasValue(int colIndex) { - return reader.hasValue(colIndex); - } - - @Override - public boolean hasValue(String colName) { - return reader.hasValue(colName); - } - - @Override - public byte getByte(int index) { - return reader.getByte(index); - } - - @Override - public short getShort(int index) { - return reader.getShort(index); - } - - @Override - public int getInteger(int index) { - return reader.getInteger(index); - } - - @Override - public long getLong(int index) { - return reader.getLong(index); - } - - @Override - public float getFloat(int index) { - return reader.getFloat(index); - } - - @Override - public double getDouble(int index) { - return reader.getDouble(index); - } - - @Override - public boolean getBoolean(int index) { - return reader.getBoolean(index); - } - - @Override - public BigInteger getBigInteger(int index) { - return reader.getBigInteger(index); - } - - @Override - public BigDecimal getBigDecimal(int index) { - return reader.getBigDecimal(index); - } - - @Override - public Instant getInstant(int index) { - return reader.getInstant(index); - } - - @Override - public ZonedDateTime getZonedDateTime(int index) { - return reader.getZonedDateTime(index); - } - - @Override - public Duration getDuration(int index) { - return reader.getDuration(index); - } - - @Override - public TemporalAmount getTemporalAmount(int index) { - return reader.getTemporalAmount(index); - } - - @Override - public Inet4Address getInet4Address(int index) { - return reader.getInet4Address(index); - } - - @Override - public Inet6Address getInet6Address(int index) { - return reader.getInet6Address(index); - } - - @Override - public UUID getUUID(int index) { - return reader.getUUID(index); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(int index) { - return reader.getGeoPoint(index); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(int index) { - return reader.getGeoRing(index); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - return reader.getGeoPolygon(index); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - return reader.getGeoMultiPolygon(index); - } - - @Override - public List getList(int index) { - return reader.getList(index); - } - - @Override - public byte[] getByteArray(int index) { - return reader.getByteArray(index); - } - - @Override - public int[] getIntArray(int index) { - return reader.getIntArray(index); - } - - @Override - public long[] getLongArray(int index) { - return reader.getLongArray(index); - } - - @Override - public float[] getFloatArray(int index) { - return reader.getFloatArray(index); - } - - @Override - public double[] getDoubleArray(int index) { - return reader.getDoubleArray(index); - } - - @Override - public boolean[] getBooleanArray(int index) { - return reader.getBooleanArray(index); - } - - @Override - public short[] getShortArray(int index) { - return reader.getShortArray(index); - } - - @Override - public String[] getStringArray(int index) { - return reader.getStringArray(index); - } - - @Override - public Object[] getObjectArray(int index) { - return reader.getObjectArray(index); - } - - @Override - public Object[] getTuple(int index) { - return reader.getTuple(index); - } - - @Override - public Object[] getTuple(String colName) { - return reader.getTuple(colName); - } - - @Override - public byte getEnum8(String colName) { - return reader.getEnum8(colName); - } - - @Override - public byte getEnum8(int index) { - return reader.getEnum8(index); - } - - @Override - public short getEnum16(String colName) { - return reader.getEnum16(colName); - } - - @Override - public short getEnum16(int index) { - return reader.getEnum16(index); - } - - @Override - public LocalDate getLocalDate(String colName) { - return reader.getLocalDate(colName); - } - - @Override - public LocalDate getLocalDate(int index) { - return reader.getLocalDate(index); - } - - @Override - public LocalTime getLocalTime(String colName) { - return reader.getLocalTime(colName); - } - - @Override - public LocalTime getLocalTime(int index) { - return reader.getLocalTime(index); - } - - @Override - public LocalDateTime getLocalDateTime(String colName) { - return reader.getLocalDateTime(colName); - } - - @Override - public LocalDateTime getLocalDateTime(int index) { - return reader.getLocalDateTime(index); - } - - @Override - public OffsetDateTime getOffsetDateTime(String colName) { - return reader.getOffsetDateTime(colName); - } - - @Override - public OffsetDateTime getOffsetDateTime(int index) { - return reader.getOffsetDateTime(index); - } - - @Override - public Object getObject(String colName) { - return reader.readValue(colName); - } - - @Override - public Object getObject(int index) { - return reader.readValue(index); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(String colName) { - return reader.readValue(colName); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(int index) { - return reader.readValue(index); - } - - @Override - public TableSchema getSchema() { - return reader.getSchema(); - } - - @Override - public Map getValues() { - return this.getSchema().getColumns().stream().collect(Collectors.toMap( - ClickHouseColumn::getColumnName, - column -> this.getObject(column.getColumnName()))); - } -} +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.value.*; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.time.*; +import java.time.temporal.TemporalAmount; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +public class BinaryReaderBackedRecord implements GenericRecord { + + private final ClickHouseBinaryFormatReader reader; + + public BinaryReaderBackedRecord(ClickHouseBinaryFormatReader reader) { + this.reader = reader; + } + + @Override + public String getString(String colName) { + return reader.getString(colName); + } + + @Override + public byte getByte(String colName) { + return reader.getByte(colName); + } + + @Override + public short getShort(String colName) { + return reader.getShort(colName); + } + + @Override + public int getInteger(String colName) { + return reader.getInteger(colName); + } + + @Override + public long getLong(String colName) { + return reader.getLong(colName); + } + + @Override + public float getFloat(String colName) { + return reader.getFloat(colName); + } + + @Override + public double getDouble(String colName) { + return reader.getDouble(colName); + } + + @Override + public boolean getBoolean(String colName) { + return reader.getBoolean(colName); + } + + @Override + public BigInteger getBigInteger(String colName) { + return reader.getBigInteger(colName); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + return reader.getBigDecimal(colName); + } + + @Override + public Instant getInstant(String colName) { + return reader.getInstant(colName); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + return reader.getZonedDateTime(colName); + } + + @Override + public Duration getDuration(String colName) { + return reader.getDuration(colName); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return reader.getTemporalAmount(colName); + } + + @Override + public Inet4Address getInet4Address(String colName) { + return reader.getInet4Address(colName); + } + + @Override + public Inet6Address getInet6Address(String colName) { + return reader.getInet6Address(colName); + } + + @Override + public UUID getUUID(String colName) { + return reader.getUUID(colName); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + return reader.getGeoPoint(colName); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + return reader.getGeoRing(colName); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + return reader.getGeoPolygon(colName); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + return reader.getGeoMultiPolygon(colName); + } + + @Override + public List getList(String colName) { + return reader.getList(colName); + } + + @Override + public byte[] getByteArray(String colName) { + return reader.getByteArray(colName); + } + + @Override + public int[] getIntArray(String colName) { + return reader.getIntArray(colName); + } + + @Override + public long[] getLongArray(String colName) { + return reader.getLongArray(colName); + } + + @Override + public float[] getFloatArray(String colName) { + return reader.getFloatArray(colName); + } + + @Override + public double[] getDoubleArray(String colName) { + return reader.getDoubleArray(colName); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return reader.getBooleanArray(colName); + } + + @Override + public short[] getShortArray(String colName) { + return reader.getShortArray(colName); + } + + @Override + public String[] getStringArray(String colName) { + return reader.getStringArray(colName); + } + + @Override + public Object[] getObjectArray(String colName) { + return reader.getObjectArray(colName); + } + + @Override + public String getString(int index) { + return reader.getString(index); + } + + @Override + public boolean hasValue(int colIndex) { + return reader.hasValue(colIndex); + } + + @Override + public boolean hasValue(String colName) { + return reader.hasValue(colName); + } + + @Override + public byte getByte(int index) { + return reader.getByte(index); + } + + @Override + public short getShort(int index) { + return reader.getShort(index); + } + + @Override + public int getInteger(int index) { + return reader.getInteger(index); + } + + @Override + public long getLong(int index) { + return reader.getLong(index); + } + + @Override + public float getFloat(int index) { + return reader.getFloat(index); + } + + @Override + public double getDouble(int index) { + return reader.getDouble(index); + } + + @Override + public boolean getBoolean(int index) { + return reader.getBoolean(index); + } + + @Override + public BigInteger getBigInteger(int index) { + return reader.getBigInteger(index); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return reader.getBigDecimal(index); + } + + @Override + public Instant getInstant(int index) { + return reader.getInstant(index); + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + return reader.getZonedDateTime(index); + } + + @Override + public Duration getDuration(int index) { + return reader.getDuration(index); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return reader.getTemporalAmount(index); + } + + @Override + public Inet4Address getInet4Address(int index) { + return reader.getInet4Address(index); + } + + @Override + public Inet6Address getInet6Address(int index) { + return reader.getInet6Address(index); + } + + @Override + public UUID getUUID(int index) { + return reader.getUUID(index); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + return reader.getGeoPoint(index); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + return reader.getGeoRing(index); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + return reader.getGeoPolygon(index); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + return reader.getGeoMultiPolygon(index); + } + + @Override + public List getList(int index) { + return reader.getList(index); + } + + @Override + public byte[] getByteArray(int index) { + return reader.getByteArray(index); + } + + @Override + public int[] getIntArray(int index) { + return reader.getIntArray(index); + } + + @Override + public long[] getLongArray(int index) { + return reader.getLongArray(index); + } + + @Override + public float[] getFloatArray(int index) { + return reader.getFloatArray(index); + } + + @Override + public double[] getDoubleArray(int index) { + return reader.getDoubleArray(index); + } + + @Override + public boolean[] getBooleanArray(int index) { + return reader.getBooleanArray(index); + } + + @Override + public short[] getShortArray(int index) { + return reader.getShortArray(index); + } + + @Override + public String[] getStringArray(int index) { + return reader.getStringArray(index); + } + + @Override + public Object[] getObjectArray(int index) { + return reader.getObjectArray(index); + } + + @Override + public Object[] getTuple(int index) { + return reader.getTuple(index); + } + + @Override + public Object[] getTuple(String colName) { + return reader.getTuple(colName); + } + + @Override + public byte getEnum8(String colName) { + return reader.getEnum8(colName); + } + + @Override + public byte getEnum8(int index) { + return reader.getEnum8(index); + } + + @Override + public short getEnum16(String colName) { + return reader.getEnum16(colName); + } + + @Override + public short getEnum16(int index) { + return reader.getEnum16(index); + } + + @Override + public LocalDate getLocalDate(String colName) { + return reader.getLocalDate(colName); + } + + @Override + public LocalDate getLocalDate(int index) { + return reader.getLocalDate(index); + } + + @Override + public LocalTime getLocalTime(String colName) { + return reader.getLocalTime(colName); + } + + @Override + public LocalTime getLocalTime(int index) { + return reader.getLocalTime(index); + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + return reader.getLocalDateTime(colName); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + return reader.getLocalDateTime(index); + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + return reader.getOffsetDateTime(colName); + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + return reader.getOffsetDateTime(index); + } + + @Override + public Object getObject(String colName) { + return reader.readValue(colName); + } + + @Override + public Object getObject(int index) { + return reader.readValue(index); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + return reader.readValue(colName); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + return reader.readValue(index); + } + + @Override + public TableSchema getSchema() { + return reader.getSchema(); + } + + @Override + public Map getValues() { + return this.getSchema().getColumns().stream().collect(Collectors.toMap( + ClickHouseColumn::getColumnName, + column -> this.getObject(column.getColumnName()))); + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index 07e65de39..d83dd22c0 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -1,686 +1,686 @@ -package com.clickhouse.client.api.data_formats.internal; - -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.internal.DataTypeConverter; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.client.api.query.NullValueException; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.value.ClickHouseBitmap; -import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoPointValue; -import com.clickhouse.data.value.ClickHouseGeoPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoRingValue; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.time.temporal.TemporalAmount; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Function; - -public class MapBackedRecord implements GenericRecord { - - private final Map record; - - private final TableSchema schema; - - private Map[] columnConverters; - - private DataTypeConverter dataTypeConverter; - - public MapBackedRecord(Map record, Map[] columnConverters, TableSchema schema) { - this.record = new HashMap<>(record); - this.schema = schema; - this.columnConverters = columnConverters; - this.dataTypeConverter = DataTypeConverter.INSTANCE; - } - - public T readValue(int colIndex) { - if (colIndex < 1 || colIndex > schema.getColumns().size()) { - throw new ClientException("Column index out of bounds: " + colIndex); - } - - return (T) record.get(schema.columnIndexToName(colIndex)); - } - - public T readValue(String colName) { - return (T) record.get(colName); - } - - @Override - public String getString(String colName) { - return dataTypeConverter.convertToString(readValue(colName), schema.getColumnByName(colName)); - } - - @Override - public String getString(int index) { - return getString(schema.columnIndexToName(index)); - } - - private T readNumberValue(String colName, NumberConverter.NumberType targetType) { - int colIndex = schema.nameToIndex(colName); - - Function converter = (Function) columnConverters[colIndex].get(targetType); - if (converter != null) { - Object value = readValue(colName); - if (value == null) { - if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { - return null; - } - throw new NullValueException("Column " + colName + " has null value and it cannot be cast to " + - targetType.getTypeName()); - } - return (T) converter.apply(value); - } else { - String columnTypeName = schema.getColumnByName(colName).getDataType().name(); - throw new ClientException("Column '" + colName + "' of type " + columnTypeName + - " cannot be converted to '" + targetType.getTypeName() + "' value"); - } - } - - @Override - public byte getByte(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Byte); - } - - @Override - public short getShort(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Short); - } - - @Override - public int getInteger(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Int); - } - - @Override - public long getLong(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Long); - } - - @Override - public float getFloat(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Float); - } - - @Override - public double getDouble(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Double); - } - - @Override - public boolean getBoolean(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Boolean); - } - - @Override - public BigInteger getBigInteger(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.BigInteger); - } - - @Override - public BigDecimal getBigDecimal(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.BigDecimal); - } - - @Override - public Instant getInstant(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - int colIndex = column.getColumnIndex(); - switch (column.getValueDataType()) { - case Date: - case Date32: - LocalDate date = getLocalDate(colIndex); - return date == null ? null : Instant.from(date); - case Time: - case Time64: - LocalDateTime time = getLocalDateTime(colName); - return time == null ? null : time.toInstant(ZoneOffset.UTC); - case DateTime: - case DateTime64: - case DateTime32: - ZonedDateTime zdt = getZonedDateTime(colName); - return zdt == null ? null : zdt.toInstant(); - case Dynamic: - case Variant: - Object value = readValue(colName); - Instant instant = AbstractBinaryFormatReader.objectToInstant(value); - if (value == null || instant != null) { - return instant; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); - } - - @Override - public ZonedDateTime getZonedDateTime(String colName) { - return getZonedDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public Duration getDuration(String colName) { - TemporalAmount temporalAmount = readValue(colName); - return temporalAmount == null ? null : Duration.from(temporalAmount); - } - - @Override - public TemporalAmount getTemporalAmount(String colName) { - return readValue(colName); - } - - @Override - public Inet4Address getInet4Address(String colName) { - Object val = readValue(colName); - return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); - } - - @Override - public Inet6Address getInet6Address(String colName) { - Object val = readValue(colName); - return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); - } - - @Override - public UUID getUUID(String colName) { - return readValue(colName); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); - } - - - @Override - public List getList(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).asList(); - } else if (value instanceof List) { - return (List) value; - } else { - throw new ClientException("Column is not of array type"); - } - } - - - private T getPrimitiveArray(String colName) { - BinaryStreamReader.ArrayValue array = readValue(colName); - if (array == null) { - return null; - } - if (array.itemType.isPrimitive()) { - return (T) array.array; - } else { - throw new ClientException("Array is not of primitive type"); - } - } - - @Override - public byte[] getByteArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public int[] getIntArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public long[] getLongArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public float[] getFloatArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public double[] getDoubleArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public boolean[] getBooleanArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public short[] getShortArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public String[] getStringArray(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - if (array.itemType == String.class) { - return (String[]) array.getArray(); - } else if (array.itemType == BinaryStreamReader.EnumValue.class) { - BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); - return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); - } else { - throw new ClientException("Not an array of strings"); - } - } - throw new ClientException("Column is not of array type"); - } - - @Override - public boolean hasValue(int colIndex) { - return hasValue(schema.columnIndexToName(colIndex)); - } - - @Override - public boolean hasValue(String colName) { - return record.containsKey(colName); - } - - @Override - public byte getByte(int index) { - return getByte(schema.columnIndexToName(index)); - } - - @Override - public short getShort(int index) { - return getShort(schema.columnIndexToName(index)); - } - - @Override - public int getInteger(int index) { - return getInteger(schema.columnIndexToName(index)); - } - - @Override - public long getLong(int index) { - return getLong(schema.columnIndexToName(index)); - } - - @Override - public float getFloat(int index) { - return getFloat(schema.columnIndexToName(index)); - } - - @Override - public double getDouble(int index) { - return getDouble(schema.columnIndexToName(index)); - } - - @Override - public boolean getBoolean(int index) { - return getBoolean(schema.columnIndexToName(index)); - } - - @Override - public BigInteger getBigInteger(int index) { - return getBigInteger(schema.columnIndexToName(index)); - } - - @Override - public BigDecimal getBigDecimal(int index) { - return getBigDecimal(schema.columnIndexToName(index)); - } - - @Override - public Instant getInstant(int index) { - return getInstant(schema.columnIndexToName(index)); - } - - @Override - public ZonedDateTime getZonedDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch (column.getValueDataType()) { - case DateTime: - case DateTime64: - case DateTime32: - return readValue(index); - case Dynamic: - case Variant: - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof ZonedDateTime) { - return (ZonedDateTime) value; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); - } - - @Override - public Duration getDuration(int index) { - return getDuration(schema.columnIndexToName(index)); - } - - @Override - public TemporalAmount getTemporalAmount(int index) { - return readValue(index); - } - - @Override - public Inet4Address getInet4Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); - } - - @Override - public Inet6Address getInet6Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); - } - - @Override - public UUID getUUID(int index) { - return readValue(index); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(int index) { - return getGeoPoint(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(int index) { - return getGeoRing(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - return getGeoPolygon(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - return getGeoMultiPolygon(schema.columnIndexToName(index)); - } - - @Override - public List getList(int index) { - return getList(schema.columnIndexToName(index)); - } - - @Override - public byte[] getByteArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public int[] getIntArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public long[] getLongArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public float[] getFloatArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public double[] getDoubleArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public boolean[] getBooleanArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public short[] getShortArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public String[] getStringArray(int index) { - return getStringArray(schema.columnIndexToName(index)); - } - - @Override - public Object[] getObjectArray(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); - } else if (value instanceof List) { - return ((List) value).toArray(new Object[0]); - } - throw new ClientException("Column is not of array type"); - } - - @Override - public Object[] getObjectArray(int index) { - return getObjectArray(schema.columnIndexToName(index)); - } - - @Override - public Object[] getTuple(int index) { - return readValue(index); - } - - @Override - public Object[] getTuple(String colName) { - return readValue(colName); - } - - @Override - public byte getEnum8(String colName) { - Object val = readValue(colName); - if (val == null) { - throw new NullValueException("Column " + colName + " has null value and it cannot be cast to byte"); - } - if (val instanceof BinaryStreamReader.EnumValue) { - return ((BinaryStreamReader.EnumValue) val).byteValue(); - } - return (byte) val; - } - - @Override - public byte getEnum8(int index) { - return getEnum8(schema.columnIndexToName(index)); - } - - @Override - public short getEnum16(String colName) { - Object val = readValue(colName); - if (val == null) { - throw new NullValueException("Column " + colName + " has null value and it cannot be cast to short"); - } - if (val instanceof BinaryStreamReader.EnumValue) { - return ((BinaryStreamReader.EnumValue) val).shortValue(); - } - return (short) val; - } - - @Override - public short getEnum16(int index) { - return getEnum16(schema.columnIndexToName(index)); - } - - @Override - public LocalDate getLocalDate(int index) { - return getLocalDate(schema.columnIndexToName(index)); - } - - @Override - public LocalDate getLocalDate(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case Date: - case Date32: - return (LocalDate) getObject(colName); - case DateTime: - case DateTime32: - case DateTime64: - LocalDateTime dt = getLocalDateTime(colName); - return dt == null ? null : dt.toLocalDate(); - case Dynamic: - case Variant: - Object value = getObject(colName); - LocalDate localDate = AbstractBinaryFormatReader.objectToLocalDate(value); - if (value == null || localDate != null) { - return localDate; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); - } - - @Override - public LocalTime getLocalTime(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case Time: - case Time64: - LocalDateTime val = (LocalDateTime) getObject(colName); - return val == null ? null : val.toLocalTime(); - case DateTime: - case DateTime32: - case DateTime64: - LocalDateTime dt = getLocalDateTime(colName); - return dt == null ? null : dt.toLocalTime(); - case Dynamic: - case Variant: - Object value = getObject(colName); - LocalTime localTime = AbstractBinaryFormatReader.objectToLocalTime(value); - if (value == null || localTime != null) { - return localTime; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); - } - - @Override - public LocalTime getLocalTime(int index) { - return getLocalTime(schema.columnIndexToName(index)); - } - - @Override - public LocalDateTime getLocalDateTime(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case Time: - case Time64: - // Types present wide range of value so LocalDateTime let to access to actual value - return (LocalDateTime) getObject(colName); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime val = (ZonedDateTime) readValue(colName); - return val == null ? null : val.toLocalDateTime(); - case Dynamic: - case Variant: - Object value = getObject(colName); - LocalDateTime localDateTime = AbstractBinaryFormatReader.objectToLocalDateTime(value); - if (value == null || localDateTime != null) { - return localDateTime; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); - } - - @Override - public LocalDateTime getLocalDateTime(int index) { - return getLocalDateTime(schema.columnIndexToName(index)); - } - - @Override - public OffsetDateTime getOffsetDateTime(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case DateTime: - case DateTime32: - case DateTime64: - case Dynamic: - case Variant: - ZonedDateTime val = getZonedDateTime(colName); - return val == null ? null : val.toOffsetDateTime(); - default: - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDataTime"); - } - } - - @Override - public OffsetDateTime getOffsetDateTime(int index) { - return getOffsetDateTime(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(String colName) { - return readValue(colName); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(int index) { - return readValue(index); - } - - @Override - public TableSchema getSchema() { - return this.schema; - } - - @Override - public Object getObject(String colName) { - return readValue(colName); - } - - @Override - public Object getObject(int index) { - return readValue(index); - } - - @Override - public Map getValues() { - return this.record; - } -} +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.internal.DataTypeConverter; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.NullValueException; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +public class MapBackedRecord implements GenericRecord { + + private final Map record; + + private final TableSchema schema; + + private Map[] columnConverters; + + private DataTypeConverter dataTypeConverter; + + public MapBackedRecord(Map record, Map[] columnConverters, TableSchema schema) { + this.record = new HashMap<>(record); + this.schema = schema; + this.columnConverters = columnConverters; + this.dataTypeConverter = DataTypeConverter.INSTANCE; + } + + public T readValue(int colIndex) { + if (colIndex < 1 || colIndex > schema.getColumns().size()) { + throw new ClientException("Column index out of bounds: " + colIndex); + } + + return (T) record.get(schema.columnIndexToName(colIndex)); + } + + public T readValue(String colName) { + return (T) record.get(colName); + } + + @Override + public String getString(String colName) { + return dataTypeConverter.convertToString(readValue(colName), schema.getColumnByName(colName)); + } + + @Override + public String getString(int index) { + return getString(schema.columnIndexToName(index)); + } + + private T readNumberValue(String colName, NumberConverter.NumberType targetType) { + int colIndex = schema.nameToIndex(colName); + + Function converter = (Function) columnConverters[colIndex].get(targetType); + if (converter != null) { + Object value = readValue(colName); + if (value == null) { + if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { + return null; + } + throw new NullValueException("Column " + colName + " has null value and it cannot be cast to " + + targetType.getTypeName()); + } + return (T) converter.apply(value); + } else { + String columnTypeName = schema.getColumnByName(colName).getDataType().name(); + throw new ClientException("Column '" + colName + "' of type " + columnTypeName + + " cannot be converted to '" + targetType.getTypeName() + "' value"); + } + } + + @Override + public byte getByte(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Byte); + } + + @Override + public short getShort(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Short); + } + + @Override + public int getInteger(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Int); + } + + @Override + public long getLong(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Long); + } + + @Override + public float getFloat(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Float); + } + + @Override + public double getDouble(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Double); + } + + @Override + public boolean getBoolean(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Boolean); + } + + @Override + public BigInteger getBigInteger(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.BigInteger); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.BigDecimal); + } + + @Override + public Instant getInstant(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + int colIndex = column.getColumnIndex(); + switch (column.getValueDataType()) { + case Date: + case Date32: + LocalDate date = getLocalDate(colIndex); + return date == null ? null : Instant.from(date); + case Time: + case Time64: + LocalDateTime time = getLocalDateTime(colName); + return time == null ? null : time.toInstant(ZoneOffset.UTC); + case DateTime: + case DateTime64: + case DateTime32: + ZonedDateTime zdt = getZonedDateTime(colName); + return zdt == null ? null : zdt.toInstant(); + case Dynamic: + case Variant: + Object value = readValue(colName); + Instant instant = AbstractBinaryFormatReader.objectToInstant(value); + if (value == null || instant != null) { + return instant; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + return getZonedDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public Duration getDuration(String colName) { + TemporalAmount temporalAmount = readValue(colName); + return temporalAmount == null ? null : Duration.from(temporalAmount); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return readValue(colName); + } + + @Override + public Inet4Address getInet4Address(String colName) { + Object val = readValue(colName); + return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); + } + + @Override + public Inet6Address getInet6Address(String colName) { + Object val = readValue(colName); + return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); + } + + @Override + public UUID getUUID(String colName) { + return readValue(colName); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); + } + + + @Override + public List getList(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).asList(); + } else if (value instanceof List) { + return (List) value; + } else { + throw new ClientException("Column is not of array type"); + } + } + + + private T getPrimitiveArray(String colName) { + BinaryStreamReader.ArrayValue array = readValue(colName); + if (array == null) { + return null; + } + if (array.itemType.isPrimitive()) { + return (T) array.array; + } else { + throw new ClientException("Array is not of primitive type"); + } + } + + @Override + public byte[] getByteArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public int[] getIntArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public long[] getLongArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public float[] getFloatArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public double[] getDoubleArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public short[] getShortArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public String[] getStringArray(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType == String.class) { + return (String[]) array.getArray(); + } else if (array.itemType == BinaryStreamReader.EnumValue.class) { + BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); + return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); + } else { + throw new ClientException("Not an array of strings"); + } + } + throw new ClientException("Column is not of array type"); + } + + @Override + public boolean hasValue(int colIndex) { + return hasValue(schema.columnIndexToName(colIndex)); + } + + @Override + public boolean hasValue(String colName) { + return record.containsKey(colName); + } + + @Override + public byte getByte(int index) { + return getByte(schema.columnIndexToName(index)); + } + + @Override + public short getShort(int index) { + return getShort(schema.columnIndexToName(index)); + } + + @Override + public int getInteger(int index) { + return getInteger(schema.columnIndexToName(index)); + } + + @Override + public long getLong(int index) { + return getLong(schema.columnIndexToName(index)); + } + + @Override + public float getFloat(int index) { + return getFloat(schema.columnIndexToName(index)); + } + + @Override + public double getDouble(int index) { + return getDouble(schema.columnIndexToName(index)); + } + + @Override + public boolean getBoolean(int index) { + return getBoolean(schema.columnIndexToName(index)); + } + + @Override + public BigInteger getBigInteger(int index) { + return getBigInteger(schema.columnIndexToName(index)); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return getBigDecimal(schema.columnIndexToName(index)); + } + + @Override + public Instant getInstant(int index) { + return getInstant(schema.columnIndexToName(index)); + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch (column.getValueDataType()) { + case DateTime: + case DateTime64: + case DateTime32: + return readValue(index); + case Dynamic: + case Variant: + Object value = readValue(index); + if (value == null) { + return null; + } else if (value instanceof ZonedDateTime) { + return (ZonedDateTime) value; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); + } + + @Override + public Duration getDuration(int index) { + return getDuration(schema.columnIndexToName(index)); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return readValue(index); + } + + @Override + public Inet4Address getInet4Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); + } + + @Override + public Inet6Address getInet6Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); + } + + @Override + public UUID getUUID(int index) { + return readValue(index); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + return getGeoPoint(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + return getGeoRing(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + return getGeoPolygon(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + return getGeoMultiPolygon(schema.columnIndexToName(index)); + } + + @Override + public List getList(int index) { + return getList(schema.columnIndexToName(index)); + } + + @Override + public byte[] getByteArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public int[] getIntArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public long[] getLongArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public float[] getFloatArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public double[] getDoubleArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public boolean[] getBooleanArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public short[] getShortArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public String[] getStringArray(int index) { + return getStringArray(schema.columnIndexToName(index)); + } + + @Override + public Object[] getObjectArray(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getObjectArray(int index) { + return getObjectArray(schema.columnIndexToName(index)); + } + + @Override + public Object[] getTuple(int index) { + return readValue(index); + } + + @Override + public Object[] getTuple(String colName) { + return readValue(colName); + } + + @Override + public byte getEnum8(String colName) { + Object val = readValue(colName); + if (val == null) { + throw new NullValueException("Column " + colName + " has null value and it cannot be cast to byte"); + } + if (val instanceof BinaryStreamReader.EnumValue) { + return ((BinaryStreamReader.EnumValue) val).byteValue(); + } + return (byte) val; + } + + @Override + public byte getEnum8(int index) { + return getEnum8(schema.columnIndexToName(index)); + } + + @Override + public short getEnum16(String colName) { + Object val = readValue(colName); + if (val == null) { + throw new NullValueException("Column " + colName + " has null value and it cannot be cast to short"); + } + if (val instanceof BinaryStreamReader.EnumValue) { + return ((BinaryStreamReader.EnumValue) val).shortValue(); + } + return (short) val; + } + + @Override + public short getEnum16(int index) { + return getEnum16(schema.columnIndexToName(index)); + } + + @Override + public LocalDate getLocalDate(int index) { + return getLocalDate(schema.columnIndexToName(index)); + } + + @Override + public LocalDate getLocalDate(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case Date: + case Date32: + return (LocalDate) getObject(colName); + case DateTime: + case DateTime32: + case DateTime64: + LocalDateTime dt = getLocalDateTime(colName); + return dt == null ? null : dt.toLocalDate(); + case Dynamic: + case Variant: + Object value = getObject(colName); + LocalDate localDate = AbstractBinaryFormatReader.objectToLocalDate(value); + if (value == null || localDate != null) { + return localDate; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); + } + + @Override + public LocalTime getLocalTime(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case Time: + case Time64: + LocalDateTime val = (LocalDateTime) getObject(colName); + return val == null ? null : val.toLocalTime(); + case DateTime: + case DateTime32: + case DateTime64: + LocalDateTime dt = getLocalDateTime(colName); + return dt == null ? null : dt.toLocalTime(); + case Dynamic: + case Variant: + Object value = getObject(colName); + LocalTime localTime = AbstractBinaryFormatReader.objectToLocalTime(value); + if (value == null || localTime != null) { + return localTime; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); + } + + @Override + public LocalTime getLocalTime(int index) { + return getLocalTime(schema.columnIndexToName(index)); + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case Time: + case Time64: + // Types present wide range of value so LocalDateTime let to access to actual value + return (LocalDateTime) getObject(colName); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime val = (ZonedDateTime) readValue(colName); + return val == null ? null : val.toLocalDateTime(); + case Dynamic: + case Variant: + Object value = getObject(colName); + LocalDateTime localDateTime = AbstractBinaryFormatReader.objectToLocalDateTime(value); + if (value == null || localDateTime != null) { + return localDateTime; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + return getLocalDateTime(schema.columnIndexToName(index)); + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case DateTime: + case DateTime32: + case DateTime64: + case Dynamic: + case Variant: + ZonedDateTime val = getZonedDateTime(colName); + return val == null ? null : val.toOffsetDateTime(); + default: + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDataTime"); + } + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + return getOffsetDateTime(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + return readValue(colName); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + return readValue(index); + } + + @Override + public TableSchema getSchema() { + return this.schema; + } + + @Override + public Object getObject(String colName) { + return readValue(colName); + } + + @Override + public Object getObject(int index) { + return readValue(index); + } + + @Override + public Map getValues() { + return this.record; + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BaseReaderTests.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BaseReaderTests.java index befa6ee87..b3e9f0676 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BaseReaderTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BaseReaderTests.java @@ -11,6 +11,7 @@ import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.QueryResponse; import com.clickhouse.data.ClickHouseVersion; +import com.clickhouse.data.ClickHouseDataType; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -25,6 +26,7 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.Arrays; +import java.util.Collections; import java.util.List; @Test(groups = {"integration"}) @@ -425,7 +427,125 @@ public void testReadingOffsetDateTimeFromVariant() throws Exception { OffsetDateTime actualOffsetDateTime = records.get(0).getOffsetDateTime("field"); Assert.assertEquals(actualOffsetDateTime, expectedOffsetDateTime); } - + + @Test(groups = {"integration"}) + public void testGetObjectArrayWithNullableElements() throws Exception { + final String table = "test_get_object_array_with_nullable_elements"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDefinition(table, + "id Int32", + "arr_nullable Array(Nullable(Int32))", + "arr2d_nullable Array(Array(Nullable(Int32)))")).get(); + + client.execute("INSERT INTO " + table + " VALUES (1, [1, NULL, 2], [[1, NULL], [NULL, 3]])").get(); + + try (QueryResponse response = client.query("SELECT * FROM " + table).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response); + Assert.assertNotNull(reader.next()); + + Object[] arrNullable = reader.getObjectArray("arr_nullable"); + Assert.assertNotNull(arrNullable); + Assert.assertEquals(arrNullable.length, 3); + Assert.assertEquals(arrNullable[0], 1); + Assert.assertNull(arrNullable[1]); + Assert.assertEquals(arrNullable[2], 2); + + Object[] arr2dNullable = reader.getObjectArray("arr2d_nullable"); + Assert.assertNotNull(arr2dNullable); + Assert.assertEquals(arr2dNullable.length, 2); + Assert.assertTrue(arr2dNullable[0] instanceof Object[]); + Assert.assertTrue(arr2dNullable[1] instanceof Object[]); + + Object[] inner0 = (Object[]) arr2dNullable[0]; + Assert.assertEquals(inner0.length, 2); + Assert.assertEquals(inner0[0], 1); + Assert.assertNull(inner0[1]); + + Object[] inner1 = (Object[]) arr2dNullable[1]; + Assert.assertEquals(inner1.length, 2); + Assert.assertNull(inner1[0]); + Assert.assertEquals(inner1[1], 3); + } + + List records = client.queryAll("SELECT * FROM " + table); + Assert.assertEquals(records.size(), 1); + GenericRecord record = records.get(0); + + Object[] arrNullableRecord = record.getObjectArray("arr_nullable"); + Assert.assertNotNull(arrNullableRecord); + Assert.assertEquals(arrNullableRecord.length, 3); + Assert.assertEquals(arrNullableRecord[0], 1); + Assert.assertNull(arrNullableRecord[1]); + Assert.assertEquals(arrNullableRecord[2], 2); + + Object[] arr2dNullableRecord = record.getObjectArray("arr2d_nullable"); + Assert.assertNotNull(arr2dNullableRecord); + Assert.assertEquals(arr2dNullableRecord.length, 2); + + Object[] innerRecord0 = (Object[]) arr2dNullableRecord[0]; + Assert.assertEquals(innerRecord0.length, 2); + Assert.assertEquals(innerRecord0[0], 1); + Assert.assertNull(innerRecord0[1]); + + Object[] innerRecord1 = (Object[]) arr2dNullableRecord[1]; + Assert.assertEquals(innerRecord1.length, 2); + Assert.assertNull(innerRecord1[0]); + Assert.assertEquals(innerRecord1[1], 3); + } + + @Test(groups = {"integration"}) + public void testGetObjectArrayWhenValueIsList() throws Exception { + final String table = "test_get_object_array_when_value_is_list"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDefinition(table, + "id Int32", + "arr Array(Int32)", + "arr2d Array(Array(Int32))")).get(); + client.execute("INSERT INTO " + table + " VALUES (1, [10, 20, 30], [[1, 2], [3]])").get(); + + try (Client listClient = newClient() + .typeHintMapping(Collections.singletonMap(ClickHouseDataType.Array, Object.class)) + .build()) { + try (QueryResponse response = listClient.query("SELECT * FROM " + table).get()) { + ClickHouseBinaryFormatReader reader = listClient.newBinaryFormatReader(response); + Assert.assertNotNull(reader.next()); + + Object[] arr = reader.getObjectArray("arr"); + Assert.assertNotNull(arr); + Assert.assertEquals(arr.length, 3); + Assert.assertEquals(arr[0], 10); + Assert.assertEquals(arr[1], 20); + Assert.assertEquals(arr[2], 30); + + Object[] arr2d = reader.getObjectArray("arr2d"); + Assert.assertNotNull(arr2d); + Assert.assertEquals(arr2d.length, 2); + Assert.assertTrue(arr2d[0] instanceof List); + Assert.assertTrue(arr2d[1] instanceof List); + Assert.assertEquals((List) arr2d[0], Arrays.asList(1, 2)); + Assert.assertEquals((List) arr2d[1], Collections.singletonList(3)); + } + + List records = listClient.queryAll("SELECT * FROM " + table); + Assert.assertEquals(records.size(), 1); + + Object[] arrRecord = records.get(0).getObjectArray("arr"); + Assert.assertNotNull(arrRecord); + Assert.assertEquals(arrRecord.length, 3); + Assert.assertEquals(arrRecord[0], 10); + Assert.assertEquals(arrRecord[1], 20); + Assert.assertEquals(arrRecord[2], 30); + + Object[] arr2dRecord = records.get(0).getObjectArray("arr2d"); + Assert.assertNotNull(arr2dRecord); + Assert.assertEquals(arr2dRecord.length, 2); + Assert.assertTrue(arr2dRecord[0] instanceof List); + Assert.assertTrue(arr2dRecord[1] instanceof List); + Assert.assertEquals((List) arr2dRecord[0], Arrays.asList(1, 2)); + Assert.assertEquals((List) arr2dRecord[1], Collections.singletonList(3)); + } + } + public static String tableDefinition(String table, String... columns) { StringBuilder sb = new StringBuilder(); diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecordTest.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecordTest.java new file mode 100644 index 000000000..a73b36f7b --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecordTest.java @@ -0,0 +1,185 @@ +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.BaseIntegrationTest; +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseProtocol; +import com.clickhouse.client.ClickHouseServerForTest; +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.Records; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.math.BigInteger; +import java.time.ZonedDateTime; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +@Test(groups = {"integration"}) +public class BinaryReaderBackedRecordTest extends BaseIntegrationTest { + + private Client client; + + @BeforeMethod(groups = {"integration"}) + public void setUp() { + client = newClient().build(); + } + + @AfterMethod(groups = {"integration"}) + public void tearDown() { + if (client != null) { + client.close(); + } + } + + private Client.Builder newClient() { + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + return new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isCloud()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()); + } + + @Test(groups = {"integration"}) + public void testGetObjectArray() throws Exception { + final String table = "test_binary_reader_backed_get_object_array"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute("CREATE TABLE " + table + " (" + + "rowId Int32, " + + "uint64_arr Array(UInt64), " + + "enum_arr Array(Enum8('abc' = 1, 'cde' = 2, 'xyz' = 3)), " + + "dt_arr Array(DateTime('UTC')), " + + "str_arr Array(String), " + + "int_arr Array(Int32), " + + "arr2d Array(Array(Int64)), " + + "arr3d Array(Array(Array(Int32)))" + + ") Engine = MergeTree ORDER BY rowId").get(); + + client.execute("INSERT INTO " + table + " VALUES " + + "(1, " + + "[100, 200], " + + "['abc', 'cde'], " + + "['2030-10-09 08:07:06', '2031-10-09 08:07:06'], " + + "['hello', 'world'], " + + "[10, 20, 30], " + + "[[1, 2, 3], [4, 5]], " + + "[[[1, 2], [3]], [[4, 5, 6]]]" + + ")").get(); + + List records = new ArrayList<>(); + try (Records rs = client.queryRecords("SELECT * FROM " + table + " ORDER BY rowId").get(10, TimeUnit.SECONDS)) { + for (GenericRecord record : rs) { + records.add(record); + } + } + + Assert.assertEquals(records.size(), 1); + GenericRecord row = records.get(0); + Assert.assertTrue(row instanceof BinaryReaderBackedRecord); + + // Array(UInt64) -> getObjectArray returns BigInteger[] + Object[] uint64Arr = row.getObjectArray("uint64_arr"); + Assert.assertNotNull(uint64Arr); + Assert.assertEquals(uint64Arr.length, 2); + Assert.assertEquals(uint64Arr[0], BigInteger.valueOf(100)); + Assert.assertEquals(uint64Arr[1], BigInteger.valueOf(200)); + + // Array(Enum8) -> getObjectArray returns EnumValue[] + Object[] enumArr = row.getObjectArray("enum_arr"); + Assert.assertNotNull(enumArr); + Assert.assertEquals(enumArr.length, 2); + Assert.assertEquals(enumArr[0].toString(), "abc"); + Assert.assertEquals(enumArr[1].toString(), "cde"); + + // Array(DateTime) -> getObjectArray returns ZonedDateTime[] + Object[] dtArr = row.getObjectArray("dt_arr"); + Assert.assertNotNull(dtArr); + Assert.assertEquals(dtArr.length, 2); + Assert.assertTrue(dtArr[0] instanceof ZonedDateTime); + ZonedDateTime zdt1 = (ZonedDateTime) dtArr[0]; + Assert.assertEquals(zdt1.getYear(), 2030); + Assert.assertEquals(zdt1.getMonthValue(), 10); + + // Array(String) -> getObjectArray returns String[] + Object[] strArr = row.getObjectArray("str_arr"); + Assert.assertNotNull(strArr); + Assert.assertEquals(strArr[0], "hello"); + Assert.assertEquals(strArr[1], "world"); + + // Array(Int32) -> getObjectArray returns boxed Integer[] + Object[] intArr = row.getObjectArray("int_arr"); + Assert.assertNotNull(intArr); + Assert.assertEquals(intArr.length, 3); + Assert.assertEquals(intArr[0], 10); + Assert.assertEquals(intArr[1], 20); + Assert.assertEquals(intArr[2], 30); + + // Array(Array(Int64)) 2D -> getObjectArray returns nested Object[] + Object[] arr2d = row.getObjectArray("arr2d"); + Assert.assertNotNull(arr2d); + Assert.assertEquals(arr2d.length, 2); + Assert.assertTrue(arr2d[0] instanceof Object[]); + Object[] inner0 = (Object[]) arr2d[0]; + Assert.assertEquals(inner0.length, 3); + Assert.assertEquals(inner0[0], 1L); + Assert.assertEquals(inner0[1], 2L); + Assert.assertEquals(inner0[2], 3L); + + // Array(Array(Array(Int32))) 3D -> getObjectArray returns 3-level nested Object[] + Object[] arr3d = row.getObjectArray("arr3d"); + Assert.assertNotNull(arr3d); + Assert.assertEquals(arr3d.length, 2); + Object[] dim1_0 = (Object[]) arr3d[0]; + Assert.assertEquals(dim1_0.length, 2); + Object[] dim2_0_0 = (Object[]) dim1_0[0]; + Assert.assertEquals(dim2_0_0[0], 1); + Assert.assertEquals(dim2_0_0[1], 2); + } + + @Test(groups = {"integration"}) + public void testGetObjectArrayEmptyAndEdgeCases() throws Exception { + final String table = "test_binary_reader_backed_get_object_array_empty"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute("CREATE TABLE " + table + " (" + + "rowId Int32, " + + "empty_arr Array(Int32), " + + "single_arr Array(String), " + + "arr2d_empty Array(Array(Int64))" + + ") Engine = MergeTree ORDER BY rowId").get(); + + client.execute("INSERT INTO " + table + " VALUES (1, [], ['single'], [[]])").get(); + + List records = new ArrayList<>(); + try (Records rs = client.queryRecords("SELECT * FROM " + table).get(10, TimeUnit.SECONDS)) { + for (GenericRecord record : rs) { + records.add(record); + } + } + + Assert.assertEquals(records.size(), 1); + GenericRecord row = records.get(0); + + // Empty array + Object[] emptyArr = row.getObjectArray("empty_arr"); + Assert.assertNotNull(emptyArr); + Assert.assertEquals(emptyArr.length, 0); + + // Single-element array + Object[] singleArr = row.getObjectArray("single_arr"); + Assert.assertNotNull(singleArr); + Assert.assertEquals(singleArr.length, 1); + Assert.assertEquals(singleArr[0], "single"); + + // 2D with inner empty: [[]] + Object[] arr2dEmpty = row.getObjectArray("arr2d_empty"); + Assert.assertNotNull(arr2dEmpty); + Assert.assertEquals(arr2dEmpty.length, 1); + Assert.assertTrue(arr2dEmpty[0] instanceof Object[]); + Assert.assertEquals(((Object[]) arr2dEmpty[0]).length, 0); + } +} \ No newline at end of file diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java index 6c0071245..4a7a07059 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -1466,6 +1466,63 @@ public void testGetObjectArrayMethods() throws Exception { Assert.assertEquals(((Object[]) arr3dInner[0]).length, 0); } + @Test(groups = {"integration"}) + public void testGetStringArrayAndGetObjectArrayWhenValueIsList() throws Exception { + final String table = "test_get_string_array_and_object_array_when_value_is_list"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute("CREATE TABLE " + table + " (" + + "rowId Int32, " + + "str_arr Array(String), " + + "arr2d Array(Array(Int32))" + + ") Engine = MergeTree ORDER BY rowId").get(); + + client.execute("INSERT INTO " + table + " VALUES " + + "(1, ['hello', 'world'], [[1, 2], [3]])").get(); + + try (Client listClient = newClient() + .typeHintMapping(Collections.singletonMap(ClickHouseDataType.Array, Object.class)) + .build()) { + // Reader path: arrays are decoded as List due to Array -> Object type hint mapping. + try (QueryResponse response = listClient.query("SELECT * FROM " + table).get()) { + ClickHouseBinaryFormatReader reader = listClient.newBinaryFormatReader(response); + Assert.assertNotNull(reader.next()); + + Object[] strObjectArr = reader.getObjectArray("str_arr"); + Assert.assertNotNull(strObjectArr); + Assert.assertEquals(strObjectArr, new Object[] {"hello", "world"}); + + Object[] arr2dObjectArr = reader.getObjectArray("arr2d"); + Assert.assertNotNull(arr2dObjectArr); + Assert.assertEquals(arr2dObjectArr.length, 2); + Assert.assertTrue(arr2dObjectArr[0] instanceof List); + Assert.assertTrue(arr2dObjectArr[1] instanceof List); + Assert.assertEquals((List) arr2dObjectArr[0], Arrays.asList(1, 2)); + Assert.assertEquals((List) arr2dObjectArr[1], Collections.singletonList(3)); + + Assert.expectThrows(ClientException.class, () -> reader.getStringArray("str_arr")); + } + + // queryAll path (MapBackedRecord): also list-backed values. + List records = listClient.queryAll("SELECT * FROM " + table + " ORDER BY rowId"); + Assert.assertEquals(records.size(), 1); + + GenericRecord row = records.get(0); + Object[] strObjectArr = row.getObjectArray("str_arr"); + Assert.assertNotNull(strObjectArr); + Assert.assertEquals(strObjectArr, new Object[] {"hello", "world"}); + + Object[] arr2dObjectArr = row.getObjectArray("arr2d"); + Assert.assertNotNull(arr2dObjectArr); + Assert.assertEquals(arr2dObjectArr.length, 2); + Assert.assertTrue(arr2dObjectArr[0] instanceof List); + Assert.assertTrue(arr2dObjectArr[1] instanceof List); + Assert.assertEquals((List) arr2dObjectArr[0], Arrays.asList(1, 2)); + Assert.assertEquals((List) arr2dObjectArr[1], Collections.singletonList(3)); + + Assert.expectThrows(ClientException.class, () -> row.getStringArray("str_arr")); + } + } + public static String tableDefinition(String table, String... columns) { StringBuilder sb = new StringBuilder(); sb.append("CREATE TABLE " + table + " ( "); From 9d5d0f0c918b5e7ea06a655276857fdf0c00977e Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Feb 2026 13:43:58 -0800 Subject: [PATCH 8/8] revert unix2dos done by AI. --- .../internal/AbstractBinaryFormatReader.java | 2304 ++++++++--------- .../internal/BinaryReaderBackedRecord.java | 898 +++---- .../internal/MapBackedRecord.java | 1372 +++++----- 3 files changed, 2287 insertions(+), 2287 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index c5c687edb..23b05c983 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -1,1152 +1,1152 @@ -package com.clickhouse.client.api.data_formats.internal; - -import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; -import com.clickhouse.client.api.internal.DataTypeConverter; -import com.clickhouse.client.api.internal.MapUtils; -import com.clickhouse.client.api.internal.ServerSettings; -import com.clickhouse.client.api.metadata.NoSuchColumnException; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.NullValueException; -import com.clickhouse.client.api.query.QuerySettings; -import com.clickhouse.client.api.serde.POJOFieldDeserializer; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.ClickHouseDataType; -import com.clickhouse.data.value.ClickHouseBitmap; -import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoPointValue; -import com.clickhouse.data.value.ClickHouseGeoPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoRingValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.lang.ref.WeakReference; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.net.InetAddress; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.time.temporal.TemporalAmount; -import java.util.AbstractMap; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TimeZone; -import java.util.UUID; -import java.util.function.Function; -import java.util.stream.Collectors; - -public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader { - - public static final Map> NO_TYPE_HINT_MAPPING = Collections.emptyMap(); - - private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class); - - protected InputStream input; - - protected BinaryStreamReader binaryStreamReader; - - protected DataTypeConverter dataTypeConverter; - - private TableSchema schema; - private ClickHouseColumn[] columns; - private Map[] convertions; - private boolean hasNext = true; - private boolean initialState = true; // reader is in initial state, no records have been read yet - - protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map> defaultTypeHintMap) { - this.input = inputStream; - Map settings = querySettings == null ? Collections.emptyMap() : querySettings.getAllSettings(); - Boolean useServerTimeZone = (Boolean) settings.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey()); - TimeZone timeZone = (useServerTimeZone == Boolean.TRUE && querySettings != null) ? - querySettings.getServerTimeZone() : - (TimeZone) settings.get(ClientConfigProperties.USE_TIMEZONE.getKey()); - if (timeZone == null) { - throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")"); - } - boolean jsonAsString = MapUtils.getFlag(settings, - ClientConfigProperties.serverSetting(ServerSettings.OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING), false); - this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString, - defaultTypeHintMap); - if (schema != null) { - setSchema(schema); - } - this.dataTypeConverter = DataTypeConverter.INSTANCE; // singleton while no need to customize conversion - } - - protected Object[] currentRecord; - protected Object[] nextRecord; - - protected boolean nextRecordEmpty = true; - - /** - * Reads next record into POJO object using set of serializers. - * There should be a serializer for each column in the record, otherwise it will silently skip a field - * It is done in such a way because it is not the reader concern. Calling code should validate this. - * - * Note: internal API - * @param deserializers - * @param obj - * @return - * @throws IOException - */ - public boolean readToPOJO(Map deserializers, Object obj ) throws IOException { - if (columns == null || columns.length == 0) { - return false; - } - - boolean firstColumn = true; - - for (ClickHouseColumn column : columns) { - try { - POJOFieldDeserializer deserializer = deserializers.get(column.getColumnName()); - if (deserializer != null) { - deserializer.setValue(obj, binaryStreamReader, column); - } else { - binaryStreamReader.skipValue(column); - } - firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { - endReached(); - return false; - } - throw e; - } catch (Exception e) { - throw new ClientException("Failed to set value of '" + column.getColumnName(), e); - } - } - return true; - } - - /** - * It is still internal method and should be used with care. - * Usually this method is called to read next record into internal object and affects hasNext() method. - * So after calling this one: - * - hasNext(), next() and get methods cannot be called - * - stream should be read with readRecord() method fully - * - * @param record - * @return - * @throws IOException - */ - public boolean readRecord(Map record) throws IOException { - if (columns == null || columns.length == 0) { - return false; - } - - boolean firstColumn = true; - for (ClickHouseColumn column : columns) { - try { - Object val = binaryStreamReader.readValue(column); - if (val != null) { - record.put(column.getColumnName(), val); - } else { - record.remove(column.getColumnName()); - } - firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { - endReached(); - return false; - } - throw e; - } - } - return true; - } - - protected boolean readRecord(Object[] record) throws IOException { - if (columns == null || columns.length == 0) { - return false; - } - - boolean firstColumn = true; - for (int i = 0; i < columns.length; i++) { - try { - Object val = binaryStreamReader.readValue(columns[i]); - if (val != null) { - record[i] = val; - } else { - record[i] = null; - } - firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { - endReached(); - return false; - } - throw e; - } - } - return true; - } - - @SuppressWarnings("unchecked") - @Override - public T readValue(int colIndex) { - if (colIndex < 1 || colIndex > getSchema().getColumns().size()) { - throw new ClientException("Column index out of bounds: " + colIndex); - } - return (T) currentRecord[colIndex - 1]; - } - - @SuppressWarnings("unchecked") - @Override - public T readValue(String colName) { - return (T) currentRecord[getSchema().nameToIndex(colName)]; - } - - @Override - public boolean hasNext() { - if (initialState) { - readNextRecord(); - } - - return hasNext; - } - - - protected void readNextRecord() { - initialState = false; - try { - nextRecordEmpty = true; - if (!readRecord(nextRecord)) { - endReached(); - } else { - nextRecordEmpty = false; - } - } catch (IOException e) { - endReached(); - throw new ClientException("Failed to read next row", e); - } - } - - @Override - public Map next() { - if (!hasNext) { - return null; - } - - if (!nextRecordEmpty) { - Object[] tmp = currentRecord; - currentRecord = nextRecord; - nextRecord = tmp; - readNextRecord(); - return new RecordWrapper(currentRecord, schema); - } else { - try { - if (readRecord(currentRecord)) { - readNextRecord(); - return new RecordWrapper(currentRecord, schema); - } else { - currentRecord = null; - return null; - } - } catch (IOException e) { - endReached(); - throw new ClientException("Failed to read row", e); - } - } - } - - protected void endReached() { - initialState = false; - hasNext = false; - } - - protected void setSchema(TableSchema schema) { - this.schema = schema; - this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY); - this.convertions = new Map[columns.length]; - - this.currentRecord = new Object[columns.length]; - this.nextRecord = new Object[columns.length]; - - for (int i = 0; i < columns.length; i++) { - ClickHouseColumn column = columns[i]; - ClickHouseDataType columnDataType = column.getDataType(); - if (columnDataType.equals(ClickHouseDataType.SimpleAggregateFunction)){ - columnDataType = column.getNestedColumns().get(0).getDataType(); - } - switch (columnDataType) { - case Int8: - case Int16: - case UInt8: - case Int32: - case UInt16: - case Int64: - case UInt32: - case Int128: - case UInt64: - case Int256: - case UInt128: - case UInt256: - case Float32: - case Float64: - case Decimal: - case Decimal32: - case Decimal64: - case Decimal128: - case Decimal256: - case Bool: - case String: - case Enum8: - case Enum16: - case Variant: - case Dynamic: - this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; - break; - default: - this.convertions[i] = Collections.emptyMap(); - } - } - } - - public Map[] getConvertions() { - return convertions; - } - - @Override - public TableSchema getSchema() { - return schema; - } - - @Override - public String getString(String colName) { - return getString(schema.nameToColumnIndex(colName)); - } - - @Override - public String getString(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - Object value; - switch (column.getValueDataType()) { - case Date: - case Date32: - value = getLocalDate(index); - break; - case Time: - case Time64: - value = getLocalTime(index); - break; - case DateTime: - case DateTime32: - case DateTime64: - value = getLocalDateTime(index); - break; - default: - value = readValue(index); - } - - return dataTypeConverter.convertToString(value, column); - } - - @SuppressWarnings("unchecked") - private T readNumberValue(int index, NumberConverter.NumberType targetType) { - int colIndex = index - 1; - Function converter = (Function) convertions[colIndex].get(targetType); - if (converter != null) { - Object value = readValue(index); - if (value == null) { - if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { - return null; - } - throw new NullValueException("Column at index " + index + " has null value and it cannot be cast to " + - targetType.getTypeName()); - } - return (T) converter.apply(value); - } else { - throw new ClientException("Column at index " + index + " " + columns[colIndex].getDataType().name() + - " cannot be converted to " + targetType.getTypeName()); - } - } - - @Override - public byte getByte(String colName) { - return getByte(schema.nameToColumnIndex(colName)); - } - - @Override - public short getShort(String colName) { - return getShort(schema.nameToColumnIndex(colName)); - } - - @Override - public int getInteger(String colName) { - return getInteger(schema.nameToColumnIndex(colName)); - } - - @Override - public long getLong(String colName) { - return getLong(schema.nameToColumnIndex(colName)); - } - - @Override - public float getFloat(String colName) { - return getFloat(schema.nameToColumnIndex(colName)); - } - - @Override - public double getDouble(String colName) { - return getDouble(schema.nameToColumnIndex(colName)); - } - - @Override - public boolean getBoolean(String colName) { - return getBoolean(schema.nameToColumnIndex(colName)); - } - - @Override - public BigInteger getBigInteger(String colName) { - return getBigInteger(schema.nameToColumnIndex(colName)); - } - - @Override - public BigDecimal getBigDecimal(String colName) { - return getBigDecimal(schema.nameToColumnIndex(colName)); - } - - @Override - public Instant getInstant(String colName) { - return getInstant(getSchema().nameToColumnIndex(colName)); - } - - @Override - public ZonedDateTime getZonedDateTime(String colName) { - return getZonedDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public Duration getDuration(String colName) { - return getDuration(schema.nameToColumnIndex(colName)); - } - - @Override - public TemporalAmount getTemporalAmount(String colName) { - return getTemporalAmount(schema.nameToColumnIndex(colName)); - } - - @Override - public Inet4Address getInet4Address(String colName) { - return getInet4Address(schema.nameToColumnIndex(colName)); - } - - @Override - public Inet6Address getInet6Address(String colName) { - return getInet6Address(schema.nameToColumnIndex(colName)); - } - - @Override - public UUID getUUID(String colName) { - return getUUID(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(String colName) { - return getGeoPoint(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(String colName) { - return getGeoRing(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { - return getGeoPolygon(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { - return getGeoMultiPolygon(schema.nameToColumnIndex(colName)); - } - - - @Override - public List getList(String colName) { - return getList(schema.nameToColumnIndex(colName)); - } - - - @SuppressWarnings("unchecked") - private T getPrimitiveArray(int index, Class componentType) { - try { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - if (array.itemType.isPrimitive()) { - return (T) array.array; - } else { - throw new ClientException("Array is not of primitive type"); - } - } else if (value instanceof List) { - List list = (List) value; - Object array = Array.newInstance(componentType, list.size()); - for (int i = 0; i < list.size(); i++) { - Array.set(array, i, list.get(i)); - } - return (T)array; - } else if (componentType == byte.class) { - if (value instanceof String) { - return (T) ((String) value).getBytes(StandardCharsets.UTF_8); - } else if (value instanceof InetAddress) { - return (T) ((InetAddress) value).getAddress(); - } - } - throw new ClientException("Column is not of array type"); - } catch (ClassCastException e) { - throw new ClientException("Column is not of array type", e); - } - } - - @Override - public byte[] getByteArray(String colName) { - return getByteArray(schema.nameToColumnIndex(colName)); - } - - @Override - public int[] getIntArray(String colName) { - return getIntArray(schema.nameToColumnIndex(colName)); - } - - @Override - public long[] getLongArray(String colName) { - return getLongArray(schema.nameToColumnIndex(colName)); - } - - @Override - public float[] getFloatArray(String colName) { - return getFloatArray(schema.nameToColumnIndex(colName)); - } - - @Override - public double[] getDoubleArray(String colName) { - return getDoubleArray(schema.nameToColumnIndex(colName)); - } - - @Override - public boolean[] getBooleanArray(String colName) { - return getBooleanArray(schema.nameToColumnIndex(colName)); - } - - @Override - public short[] getShortArray(String colName) { - return getShortArray(schema.nameToColumnIndex(colName)); - } - - @Override - public String[] getStringArray(String colName) { - return getStringArray(schema.nameToColumnIndex(colName)); - } - - @Override - public Object[] getObjectArray(String colName) { - return getObjectArray(schema.nameToColumnIndex(colName)); - } - - @Override - public boolean hasValue(int colIndex) { - return currentRecord[colIndex - 1] != null; - } - - @Override - public boolean hasValue(String colName) { - return hasValue(schema.nameToColumnIndex(colName)); - } - - @Override - public byte getByte(int index) { - return readNumberValue(index, NumberConverter.NumberType.Byte); - } - - @Override - public short getShort(int index) { - return readNumberValue(index, NumberConverter.NumberType.Short); - } - - @Override - public int getInteger(int index) { - return readNumberValue(index, NumberConverter.NumberType.Int); - } - - @Override - public long getLong(int index) { - return readNumberValue(index, NumberConverter.NumberType.Long); - } - - @Override - public float getFloat(int index) { - return readNumberValue(index, NumberConverter.NumberType.Float); - } - - @Override - public double getDouble(int index) { - return readNumberValue(index, NumberConverter.NumberType.Double); - } - - @Override - public boolean getBoolean(int index) { - return readNumberValue(index, NumberConverter.NumberType.Boolean); - } - - @Override - public BigInteger getBigInteger(int index) { - return readNumberValue(index, NumberConverter.NumberType.BigInteger); - } - - @Override - public BigDecimal getBigDecimal(int index) { - return readNumberValue(index, NumberConverter.NumberType.BigDecimal); - } - - @Override - public Instant getInstant(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch (column.getValueDataType()) { - case Date: - case Date32: - LocalDate date = getLocalDate(index); - return date == null ? null : date.atStartOfDay(ZoneId.of("UTC")).toInstant(); - case Time: - case Time64: - LocalDateTime dt = getLocalDateTime(index); - return dt == null ? null : dt.toInstant(ZoneOffset.UTC); - case DateTime: - case DateTime64: - case DateTime32: - ZonedDateTime zdt = readValue(index); - return zdt.toInstant(); - case Dynamic: - case Variant: - Object value = readValue(index); - Instant instant = objectToInstant(value); - if (value == null || instant != null) { - return instant; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); - } - - static Instant objectToInstant(Object value) { - if (value instanceof LocalDateTime) { - LocalDateTime dateTime = (LocalDateTime) value; - return Instant.from(dateTime.atZone(ZoneId.of("UTC"))); - } else if (value instanceof ZonedDateTime) { - ZonedDateTime dateTime = (ZonedDateTime) value; - return dateTime.toInstant(); - } - return null; - } - - @Override - public ZonedDateTime getZonedDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch (column.getValueDataType()) { - case DateTime: - case DateTime64: - case DateTime32: - return readValue(index); - case Dynamic: - case Variant: - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof ZonedDateTime) { - return (ZonedDateTime) value; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); - } - - @Override - public Duration getDuration(int index) { - TemporalAmount temporalAmount = getTemporalAmount(index); - return temporalAmount == null ? null : Duration.from(temporalAmount); - } - - @Override - public TemporalAmount getTemporalAmount(int index) { - return readValue(index); - } - - @Override - public Inet4Address getInet4Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); - } - - @Override - public Inet6Address getInet6Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); - } - - @Override - public UUID getUUID(int index) { - return readValue(index); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - Object val = readValue(index); - return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); - } - - @Override - public List getList(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).asList(); - } else if (value instanceof List) { - return (List) value; - } else { - throw new ClientException("Column is not of array type"); - } - } - - @Override - public byte[] getByteArray(int index) { - try { - return getPrimitiveArray(index, byte.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public int[] getIntArray(int index) { - try { - return getPrimitiveArray(index, int.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public long[] getLongArray(int index) { - try { - return getPrimitiveArray(index, long.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public float[] getFloatArray(int index) { - try { - return getPrimitiveArray(index, float.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public double[] getDoubleArray(int index) { - try { - return getPrimitiveArray(index, double.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public boolean[] getBooleanArray(int index) { - try { - return getPrimitiveArray(index, boolean.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public short[] getShortArray(int index) { - try { - return getPrimitiveArray(index, short.class); - } catch (ClassCastException | IllegalArgumentException e) { - throw new ClientException("Value cannot be converted to an array of primitives", e); - } - } - - @Override - public String[] getStringArray(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - if (array.itemType == String.class) { - return (String[]) array.getArray(); - } else if (array.itemType == BinaryStreamReader.EnumValue.class) { - BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); - return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); - } else { - throw new ClientException("Not an array of strings"); - } - } - throw new ClientException("Column is not of array type"); - } - - @Override - public Object[] getObjectArray(int index) { - Object value = readValue(index); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); - } else if (value instanceof List) { - return ((List) value).toArray(new Object[0]); - } - throw new ClientException("Column is not of array type"); - } - - @Override - public Object[] getTuple(int index) { - return readValue(index); - } - - @Override - public Object[] getTuple(String colName) { - return getTuple(schema.nameToColumnIndex(colName)); - } - - @Override - public byte getEnum8(String colName) { - return getEnum8(schema.nameToColumnIndex(colName)); - } - - @Override - public byte getEnum8(int index) { - BinaryStreamReader.EnumValue enumValue = readValue(index); - if (enumValue == null) { - throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum8 numeric value"); - } - return enumValue.byteValue(); - } - - @Override - public short getEnum16(String colName) { - return getEnum16(schema.nameToColumnIndex(colName)); - } - - @Override - public short getEnum16(int index) { - BinaryStreamReader.EnumValue enumValue = readValue(index); - if (enumValue == null) { - throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum16 numeric value"); - } - return enumValue.shortValue(); - } - - @Override - public LocalDate getLocalDate(String colName) { - return getLocalDate(schema.nameToColumnIndex(colName)); - } - - @Override - public LocalDate getLocalDate(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case Date: - case Date32: - return readValue(index); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toLocalDate(); - case Dynamic: - case Variant: - Object value = readValue(index); - LocalDate localDate = objectToLocalDate(value); - if (value == null || localDate != null) { - return localDate; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); - } - - static LocalDate objectToLocalDate(Object value) { - if (value instanceof LocalDate) { - return (LocalDate) value; - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime)value).toLocalDate(); - } else if (value instanceof LocalDateTime) { - return ((LocalDateTime)value).toLocalDate(); - } - return null; - } - - @Override - public LocalTime getLocalTime(String colName) { - return getLocalTime(schema.nameToColumnIndex(colName)); - } - - @Override - public LocalTime getLocalTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case Time: - case Time64: - LocalDateTime dt = readValue(index); - return dt == null ? null : dt.toLocalTime(); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toLocalTime(); - case Dynamic: - case Variant: - Object value = readValue(index); - LocalTime localTime = objectToLocalTime(value); - if (value == null || localTime != null) { - return localTime; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); - } - - static LocalTime objectToLocalTime(Object value) { - if (value instanceof LocalDateTime) { - return ((LocalDateTime)value).toLocalTime(); - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime)value).toLocalTime(); - } - return null; - } - - @Override - public LocalDateTime getLocalDateTime(String colName) { - return getLocalDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public LocalDateTime getLocalDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case Time: - case Time64: - return readValue(index); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toLocalDateTime(); - case Dynamic: - case Variant: - Object value = readValue(index); - LocalDateTime ldt = objectToLocalDateTime(value); - if (value == null || ldt != null) { - return ldt; - } - break; - - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); - } - - static LocalDateTime objectToLocalDateTime(Object value) { - if (value instanceof LocalDateTime) { - return (LocalDateTime) value; - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime)value).toLocalDateTime(); - } - - return null; - } - - @Override - public OffsetDateTime getOffsetDateTime(String colName) { - return getOffsetDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public OffsetDateTime getOffsetDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch(column.getValueDataType()) { - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime zdt = readValue(index); - return zdt == null ? null : zdt.toOffsetDateTime(); - case Dynamic: - case Variant: - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toOffsetDateTime(); - } - - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDateTime"); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(String colName) { - return getClickHouseBitmap(schema.nameToColumnIndex(colName)); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(int index) { - return readValue(index); - } - - @Override - public void close() throws Exception { - input.close(); - } - - private static class RecordWrapper implements Map { - - private final WeakReference recordRef; - - private final WeakReference schemaRef; - - int size; - public RecordWrapper(Object[] record, TableSchema schema) { - this.recordRef = new WeakReference<>(record); - this.schemaRef = new WeakReference<>(schema); - this.size = record.length; - } - - @Override - public int size() { - return size; - } - - @Override - public boolean isEmpty() { - return size == 0; - } - - @Override - @SuppressWarnings("ConstantConditions") - public boolean containsKey(Object key) { - if (key instanceof String) { - return recordRef.get()[schemaRef.get().nameToIndex((String)key)] != null; - } - return false; - } - - @Override - public boolean containsValue(Object value) { - for (Object obj : recordRef.get()) { - if (obj == value) { - return true; - } - } - return false; - } - - @Override - @SuppressWarnings("ConstantConditions") - public Object get(Object key) { - if (key instanceof String) { - try { - int index = schemaRef.get().nameToIndex((String) key); - if (index < size) { - return recordRef.get()[index]; - } - } catch (NoSuchColumnException e) { - return null; - } - } - - return null; - } - - @Override - public Object put(String key, Object value) { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - public Object remove(Object key) { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - public void putAll(Map m) { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - public void clear() { - throw new UnsupportedOperationException("Record is read-only"); - } - - @Override - @SuppressWarnings("ConstantConditions") - public Set keySet() { - // TODO: create a view in Schema - return schemaRef.get().getColumns().stream().map(ClickHouseColumn::getColumnName).collect(Collectors.toSet()); - } - - @Override - @SuppressWarnings("ConstantConditions") - public Collection values() { - return Arrays.asList(recordRef.get()); - } - - @Override - @SuppressWarnings("ConstantConditions") - public Set> entrySet() { - int i = 0; - Set> entrySet = new HashSet<>(); - for (ClickHouseColumn column : schemaRef.get().getColumns()) { - entrySet.add( new AbstractMap.SimpleImmutableEntry(column.getColumnName(), recordRef.get()[i++])); - } - return entrySet; - } - } -} +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.internal.DataTypeConverter; +import com.clickhouse.client.api.internal.MapUtils; +import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.metadata.NoSuchColumnException; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.NullValueException; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.client.api.serde.POJOFieldDeserializer; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.lang.ref.WeakReference; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.net.InetAddress; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TimeZone; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; + +public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader { + + public static final Map> NO_TYPE_HINT_MAPPING = Collections.emptyMap(); + + private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class); + + protected InputStream input; + + protected BinaryStreamReader binaryStreamReader; + + protected DataTypeConverter dataTypeConverter; + + private TableSchema schema; + private ClickHouseColumn[] columns; + private Map[] convertions; + private boolean hasNext = true; + private boolean initialState = true; // reader is in initial state, no records have been read yet + + protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map> defaultTypeHintMap) { + this.input = inputStream; + Map settings = querySettings == null ? Collections.emptyMap() : querySettings.getAllSettings(); + Boolean useServerTimeZone = (Boolean) settings.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey()); + TimeZone timeZone = (useServerTimeZone == Boolean.TRUE && querySettings != null) ? + querySettings.getServerTimeZone() : + (TimeZone) settings.get(ClientConfigProperties.USE_TIMEZONE.getKey()); + if (timeZone == null) { + throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")"); + } + boolean jsonAsString = MapUtils.getFlag(settings, + ClientConfigProperties.serverSetting(ServerSettings.OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING), false); + this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString, + defaultTypeHintMap); + if (schema != null) { + setSchema(schema); + } + this.dataTypeConverter = DataTypeConverter.INSTANCE; // singleton while no need to customize conversion + } + + protected Object[] currentRecord; + protected Object[] nextRecord; + + protected boolean nextRecordEmpty = true; + + /** + * Reads next record into POJO object using set of serializers. + * There should be a serializer for each column in the record, otherwise it will silently skip a field + * It is done in such a way because it is not the reader concern. Calling code should validate this. + * + * Note: internal API + * @param deserializers + * @param obj + * @return + * @throws IOException + */ + public boolean readToPOJO(Map deserializers, Object obj ) throws IOException { + if (columns == null || columns.length == 0) { + return false; + } + + boolean firstColumn = true; + + for (ClickHouseColumn column : columns) { + try { + POJOFieldDeserializer deserializer = deserializers.get(column.getColumnName()); + if (deserializer != null) { + deserializer.setValue(obj, binaryStreamReader, column); + } else { + binaryStreamReader.skipValue(column); + } + firstColumn = false; + } catch (EOFException e) { + if (firstColumn) { + endReached(); + return false; + } + throw e; + } catch (Exception e) { + throw new ClientException("Failed to set value of '" + column.getColumnName(), e); + } + } + return true; + } + + /** + * It is still internal method and should be used with care. + * Usually this method is called to read next record into internal object and affects hasNext() method. + * So after calling this one: + * - hasNext(), next() and get methods cannot be called + * - stream should be read with readRecord() method fully + * + * @param record + * @return + * @throws IOException + */ + public boolean readRecord(Map record) throws IOException { + if (columns == null || columns.length == 0) { + return false; + } + + boolean firstColumn = true; + for (ClickHouseColumn column : columns) { + try { + Object val = binaryStreamReader.readValue(column); + if (val != null) { + record.put(column.getColumnName(), val); + } else { + record.remove(column.getColumnName()); + } + firstColumn = false; + } catch (EOFException e) { + if (firstColumn) { + endReached(); + return false; + } + throw e; + } + } + return true; + } + + protected boolean readRecord(Object[] record) throws IOException { + if (columns == null || columns.length == 0) { + return false; + } + + boolean firstColumn = true; + for (int i = 0; i < columns.length; i++) { + try { + Object val = binaryStreamReader.readValue(columns[i]); + if (val != null) { + record[i] = val; + } else { + record[i] = null; + } + firstColumn = false; + } catch (EOFException e) { + if (firstColumn) { + endReached(); + return false; + } + throw e; + } + } + return true; + } + + @SuppressWarnings("unchecked") + @Override + public T readValue(int colIndex) { + if (colIndex < 1 || colIndex > getSchema().getColumns().size()) { + throw new ClientException("Column index out of bounds: " + colIndex); + } + return (T) currentRecord[colIndex - 1]; + } + + @SuppressWarnings("unchecked") + @Override + public T readValue(String colName) { + return (T) currentRecord[getSchema().nameToIndex(colName)]; + } + + @Override + public boolean hasNext() { + if (initialState) { + readNextRecord(); + } + + return hasNext; + } + + + protected void readNextRecord() { + initialState = false; + try { + nextRecordEmpty = true; + if (!readRecord(nextRecord)) { + endReached(); + } else { + nextRecordEmpty = false; + } + } catch (IOException e) { + endReached(); + throw new ClientException("Failed to read next row", e); + } + } + + @Override + public Map next() { + if (!hasNext) { + return null; + } + + if (!nextRecordEmpty) { + Object[] tmp = currentRecord; + currentRecord = nextRecord; + nextRecord = tmp; + readNextRecord(); + return new RecordWrapper(currentRecord, schema); + } else { + try { + if (readRecord(currentRecord)) { + readNextRecord(); + return new RecordWrapper(currentRecord, schema); + } else { + currentRecord = null; + return null; + } + } catch (IOException e) { + endReached(); + throw new ClientException("Failed to read row", e); + } + } + } + + protected void endReached() { + initialState = false; + hasNext = false; + } + + protected void setSchema(TableSchema schema) { + this.schema = schema; + this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY); + this.convertions = new Map[columns.length]; + + this.currentRecord = new Object[columns.length]; + this.nextRecord = new Object[columns.length]; + + for (int i = 0; i < columns.length; i++) { + ClickHouseColumn column = columns[i]; + ClickHouseDataType columnDataType = column.getDataType(); + if (columnDataType.equals(ClickHouseDataType.SimpleAggregateFunction)){ + columnDataType = column.getNestedColumns().get(0).getDataType(); + } + switch (columnDataType) { + case Int8: + case Int16: + case UInt8: + case Int32: + case UInt16: + case Int64: + case UInt32: + case Int128: + case UInt64: + case Int256: + case UInt128: + case UInt256: + case Float32: + case Float64: + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + case Bool: + case String: + case Enum8: + case Enum16: + case Variant: + case Dynamic: + this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; + break; + default: + this.convertions[i] = Collections.emptyMap(); + } + } + } + + public Map[] getConvertions() { + return convertions; + } + + @Override + public TableSchema getSchema() { + return schema; + } + + @Override + public String getString(String colName) { + return getString(schema.nameToColumnIndex(colName)); + } + + @Override + public String getString(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + Object value; + switch (column.getValueDataType()) { + case Date: + case Date32: + value = getLocalDate(index); + break; + case Time: + case Time64: + value = getLocalTime(index); + break; + case DateTime: + case DateTime32: + case DateTime64: + value = getLocalDateTime(index); + break; + default: + value = readValue(index); + } + + return dataTypeConverter.convertToString(value, column); + } + + @SuppressWarnings("unchecked") + private T readNumberValue(int index, NumberConverter.NumberType targetType) { + int colIndex = index - 1; + Function converter = (Function) convertions[colIndex].get(targetType); + if (converter != null) { + Object value = readValue(index); + if (value == null) { + if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { + return null; + } + throw new NullValueException("Column at index " + index + " has null value and it cannot be cast to " + + targetType.getTypeName()); + } + return (T) converter.apply(value); + } else { + throw new ClientException("Column at index " + index + " " + columns[colIndex].getDataType().name() + + " cannot be converted to " + targetType.getTypeName()); + } + } + + @Override + public byte getByte(String colName) { + return getByte(schema.nameToColumnIndex(colName)); + } + + @Override + public short getShort(String colName) { + return getShort(schema.nameToColumnIndex(colName)); + } + + @Override + public int getInteger(String colName) { + return getInteger(schema.nameToColumnIndex(colName)); + } + + @Override + public long getLong(String colName) { + return getLong(schema.nameToColumnIndex(colName)); + } + + @Override + public float getFloat(String colName) { + return getFloat(schema.nameToColumnIndex(colName)); + } + + @Override + public double getDouble(String colName) { + return getDouble(schema.nameToColumnIndex(colName)); + } + + @Override + public boolean getBoolean(String colName) { + return getBoolean(schema.nameToColumnIndex(colName)); + } + + @Override + public BigInteger getBigInteger(String colName) { + return getBigInteger(schema.nameToColumnIndex(colName)); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + return getBigDecimal(schema.nameToColumnIndex(colName)); + } + + @Override + public Instant getInstant(String colName) { + return getInstant(getSchema().nameToColumnIndex(colName)); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + return getZonedDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public Duration getDuration(String colName) { + return getDuration(schema.nameToColumnIndex(colName)); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return getTemporalAmount(schema.nameToColumnIndex(colName)); + } + + @Override + public Inet4Address getInet4Address(String colName) { + return getInet4Address(schema.nameToColumnIndex(colName)); + } + + @Override + public Inet6Address getInet6Address(String colName) { + return getInet6Address(schema.nameToColumnIndex(colName)); + } + + @Override + public UUID getUUID(String colName) { + return getUUID(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + return getGeoPoint(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + return getGeoRing(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + return getGeoPolygon(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + return getGeoMultiPolygon(schema.nameToColumnIndex(colName)); + } + + + @Override + public List getList(String colName) { + return getList(schema.nameToColumnIndex(colName)); + } + + + @SuppressWarnings("unchecked") + private T getPrimitiveArray(int index, Class componentType) { + try { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType.isPrimitive()) { + return (T) array.array; + } else { + throw new ClientException("Array is not of primitive type"); + } + } else if (value instanceof List) { + List list = (List) value; + Object array = Array.newInstance(componentType, list.size()); + for (int i = 0; i < list.size(); i++) { + Array.set(array, i, list.get(i)); + } + return (T)array; + } else if (componentType == byte.class) { + if (value instanceof String) { + return (T) ((String) value).getBytes(StandardCharsets.UTF_8); + } else if (value instanceof InetAddress) { + return (T) ((InetAddress) value).getAddress(); + } + } + throw new ClientException("Column is not of array type"); + } catch (ClassCastException e) { + throw new ClientException("Column is not of array type", e); + } + } + + @Override + public byte[] getByteArray(String colName) { + return getByteArray(schema.nameToColumnIndex(colName)); + } + + @Override + public int[] getIntArray(String colName) { + return getIntArray(schema.nameToColumnIndex(colName)); + } + + @Override + public long[] getLongArray(String colName) { + return getLongArray(schema.nameToColumnIndex(colName)); + } + + @Override + public float[] getFloatArray(String colName) { + return getFloatArray(schema.nameToColumnIndex(colName)); + } + + @Override + public double[] getDoubleArray(String colName) { + return getDoubleArray(schema.nameToColumnIndex(colName)); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return getBooleanArray(schema.nameToColumnIndex(colName)); + } + + @Override + public short[] getShortArray(String colName) { + return getShortArray(schema.nameToColumnIndex(colName)); + } + + @Override + public String[] getStringArray(String colName) { + return getStringArray(schema.nameToColumnIndex(colName)); + } + + @Override + public Object[] getObjectArray(String colName) { + return getObjectArray(schema.nameToColumnIndex(colName)); + } + + @Override + public boolean hasValue(int colIndex) { + return currentRecord[colIndex - 1] != null; + } + + @Override + public boolean hasValue(String colName) { + return hasValue(schema.nameToColumnIndex(colName)); + } + + @Override + public byte getByte(int index) { + return readNumberValue(index, NumberConverter.NumberType.Byte); + } + + @Override + public short getShort(int index) { + return readNumberValue(index, NumberConverter.NumberType.Short); + } + + @Override + public int getInteger(int index) { + return readNumberValue(index, NumberConverter.NumberType.Int); + } + + @Override + public long getLong(int index) { + return readNumberValue(index, NumberConverter.NumberType.Long); + } + + @Override + public float getFloat(int index) { + return readNumberValue(index, NumberConverter.NumberType.Float); + } + + @Override + public double getDouble(int index) { + return readNumberValue(index, NumberConverter.NumberType.Double); + } + + @Override + public boolean getBoolean(int index) { + return readNumberValue(index, NumberConverter.NumberType.Boolean); + } + + @Override + public BigInteger getBigInteger(int index) { + return readNumberValue(index, NumberConverter.NumberType.BigInteger); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return readNumberValue(index, NumberConverter.NumberType.BigDecimal); + } + + @Override + public Instant getInstant(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch (column.getValueDataType()) { + case Date: + case Date32: + LocalDate date = getLocalDate(index); + return date == null ? null : date.atStartOfDay(ZoneId.of("UTC")).toInstant(); + case Time: + case Time64: + LocalDateTime dt = getLocalDateTime(index); + return dt == null ? null : dt.toInstant(ZoneOffset.UTC); + case DateTime: + case DateTime64: + case DateTime32: + ZonedDateTime zdt = readValue(index); + return zdt.toInstant(); + case Dynamic: + case Variant: + Object value = readValue(index); + Instant instant = objectToInstant(value); + if (value == null || instant != null) { + return instant; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); + } + + static Instant objectToInstant(Object value) { + if (value instanceof LocalDateTime) { + LocalDateTime dateTime = (LocalDateTime) value; + return Instant.from(dateTime.atZone(ZoneId.of("UTC"))); + } else if (value instanceof ZonedDateTime) { + ZonedDateTime dateTime = (ZonedDateTime) value; + return dateTime.toInstant(); + } + return null; + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch (column.getValueDataType()) { + case DateTime: + case DateTime64: + case DateTime32: + return readValue(index); + case Dynamic: + case Variant: + Object value = readValue(index); + if (value == null) { + return null; + } else if (value instanceof ZonedDateTime) { + return (ZonedDateTime) value; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); + } + + @Override + public Duration getDuration(int index) { + TemporalAmount temporalAmount = getTemporalAmount(index); + return temporalAmount == null ? null : Duration.from(temporalAmount); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return readValue(index); + } + + @Override + public Inet4Address getInet4Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); + } + + @Override + public Inet6Address getInet6Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); + } + + @Override + public UUID getUUID(int index) { + return readValue(index); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + Object val = readValue(index); + return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); + } + + @Override + public List getList(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).asList(); + } else if (value instanceof List) { + return (List) value; + } else { + throw new ClientException("Column is not of array type"); + } + } + + @Override + public byte[] getByteArray(int index) { + try { + return getPrimitiveArray(index, byte.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public int[] getIntArray(int index) { + try { + return getPrimitiveArray(index, int.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public long[] getLongArray(int index) { + try { + return getPrimitiveArray(index, long.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public float[] getFloatArray(int index) { + try { + return getPrimitiveArray(index, float.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public double[] getDoubleArray(int index) { + try { + return getPrimitiveArray(index, double.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public boolean[] getBooleanArray(int index) { + try { + return getPrimitiveArray(index, boolean.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public short[] getShortArray(int index) { + try { + return getPrimitiveArray(index, short.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } + } + + @Override + public String[] getStringArray(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType == String.class) { + return (String[]) array.getArray(); + } else if (array.itemType == BinaryStreamReader.EnumValue.class) { + BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); + return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); + } else { + throw new ClientException("Not an array of strings"); + } + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getObjectArray(int index) { + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getTuple(int index) { + return readValue(index); + } + + @Override + public Object[] getTuple(String colName) { + return getTuple(schema.nameToColumnIndex(colName)); + } + + @Override + public byte getEnum8(String colName) { + return getEnum8(schema.nameToColumnIndex(colName)); + } + + @Override + public byte getEnum8(int index) { + BinaryStreamReader.EnumValue enumValue = readValue(index); + if (enumValue == null) { + throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum8 numeric value"); + } + return enumValue.byteValue(); + } + + @Override + public short getEnum16(String colName) { + return getEnum16(schema.nameToColumnIndex(colName)); + } + + @Override + public short getEnum16(int index) { + BinaryStreamReader.EnumValue enumValue = readValue(index); + if (enumValue == null) { + throw new NullValueException("Column at index " + index + " has null value and it cannot be converted to enum16 numeric value"); + } + return enumValue.shortValue(); + } + + @Override + public LocalDate getLocalDate(String colName) { + return getLocalDate(schema.nameToColumnIndex(colName)); + } + + @Override + public LocalDate getLocalDate(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case Date: + case Date32: + return readValue(index); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toLocalDate(); + case Dynamic: + case Variant: + Object value = readValue(index); + LocalDate localDate = objectToLocalDate(value); + if (value == null || localDate != null) { + return localDate; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); + } + + static LocalDate objectToLocalDate(Object value) { + if (value instanceof LocalDate) { + return (LocalDate) value; + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime)value).toLocalDate(); + } else if (value instanceof LocalDateTime) { + return ((LocalDateTime)value).toLocalDate(); + } + return null; + } + + @Override + public LocalTime getLocalTime(String colName) { + return getLocalTime(schema.nameToColumnIndex(colName)); + } + + @Override + public LocalTime getLocalTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case Time: + case Time64: + LocalDateTime dt = readValue(index); + return dt == null ? null : dt.toLocalTime(); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toLocalTime(); + case Dynamic: + case Variant: + Object value = readValue(index); + LocalTime localTime = objectToLocalTime(value); + if (value == null || localTime != null) { + return localTime; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); + } + + static LocalTime objectToLocalTime(Object value) { + if (value instanceof LocalDateTime) { + return ((LocalDateTime)value).toLocalTime(); + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime)value).toLocalTime(); + } + return null; + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + return getLocalDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case Time: + case Time64: + return readValue(index); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toLocalDateTime(); + case Dynamic: + case Variant: + Object value = readValue(index); + LocalDateTime ldt = objectToLocalDateTime(value); + if (value == null || ldt != null) { + return ldt; + } + break; + + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); + } + + static LocalDateTime objectToLocalDateTime(Object value) { + if (value instanceof LocalDateTime) { + return (LocalDateTime) value; + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime)value).toLocalDateTime(); + } + + return null; + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + return getOffsetDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch(column.getValueDataType()) { + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime zdt = readValue(index); + return zdt == null ? null : zdt.toOffsetDateTime(); + case Dynamic: + case Variant: + Object value = readValue(index); + if (value == null) { + return null; + } else if (value instanceof ZonedDateTime) { + return ((ZonedDateTime) value).toOffsetDateTime(); + } + + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDateTime"); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + return getClickHouseBitmap(schema.nameToColumnIndex(colName)); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + return readValue(index); + } + + @Override + public void close() throws Exception { + input.close(); + } + + private static class RecordWrapper implements Map { + + private final WeakReference recordRef; + + private final WeakReference schemaRef; + + int size; + public RecordWrapper(Object[] record, TableSchema schema) { + this.recordRef = new WeakReference<>(record); + this.schemaRef = new WeakReference<>(schema); + this.size = record.length; + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return size == 0; + } + + @Override + @SuppressWarnings("ConstantConditions") + public boolean containsKey(Object key) { + if (key instanceof String) { + return recordRef.get()[schemaRef.get().nameToIndex((String)key)] != null; + } + return false; + } + + @Override + public boolean containsValue(Object value) { + for (Object obj : recordRef.get()) { + if (obj == value) { + return true; + } + } + return false; + } + + @Override + @SuppressWarnings("ConstantConditions") + public Object get(Object key) { + if (key instanceof String) { + try { + int index = schemaRef.get().nameToIndex((String) key); + if (index < size) { + return recordRef.get()[index]; + } + } catch (NoSuchColumnException e) { + return null; + } + } + + return null; + } + + @Override + public Object put(String key, Object value) { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + public Object remove(Object key) { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + public void putAll(Map m) { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + public void clear() { + throw new UnsupportedOperationException("Record is read-only"); + } + + @Override + @SuppressWarnings("ConstantConditions") + public Set keySet() { + // TODO: create a view in Schema + return schemaRef.get().getColumns().stream().map(ClickHouseColumn::getColumnName).collect(Collectors.toSet()); + } + + @Override + @SuppressWarnings("ConstantConditions") + public Collection values() { + return Arrays.asList(recordRef.get()); + } + + @Override + @SuppressWarnings("ConstantConditions") + public Set> entrySet() { + int i = 0; + Set> entrySet = new HashSet<>(); + for (ClickHouseColumn column : schemaRef.get().getColumns()) { + entrySet.add( new AbstractMap.SimpleImmutableEntry(column.getColumnName(), recordRef.get()[i++])); + } + return entrySet; + } + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java index 4739ca83d..7faa8e0fe 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java @@ -1,449 +1,449 @@ -package com.clickhouse.client.api.data_formats.internal; - -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.value.*; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.time.*; -import java.time.temporal.TemporalAmount; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Collectors; - -public class BinaryReaderBackedRecord implements GenericRecord { - - private final ClickHouseBinaryFormatReader reader; - - public BinaryReaderBackedRecord(ClickHouseBinaryFormatReader reader) { - this.reader = reader; - } - - @Override - public String getString(String colName) { - return reader.getString(colName); - } - - @Override - public byte getByte(String colName) { - return reader.getByte(colName); - } - - @Override - public short getShort(String colName) { - return reader.getShort(colName); - } - - @Override - public int getInteger(String colName) { - return reader.getInteger(colName); - } - - @Override - public long getLong(String colName) { - return reader.getLong(colName); - } - - @Override - public float getFloat(String colName) { - return reader.getFloat(colName); - } - - @Override - public double getDouble(String colName) { - return reader.getDouble(colName); - } - - @Override - public boolean getBoolean(String colName) { - return reader.getBoolean(colName); - } - - @Override - public BigInteger getBigInteger(String colName) { - return reader.getBigInteger(colName); - } - - @Override - public BigDecimal getBigDecimal(String colName) { - return reader.getBigDecimal(colName); - } - - @Override - public Instant getInstant(String colName) { - return reader.getInstant(colName); - } - - @Override - public ZonedDateTime getZonedDateTime(String colName) { - return reader.getZonedDateTime(colName); - } - - @Override - public Duration getDuration(String colName) { - return reader.getDuration(colName); - } - - @Override - public TemporalAmount getTemporalAmount(String colName) { - return reader.getTemporalAmount(colName); - } - - @Override - public Inet4Address getInet4Address(String colName) { - return reader.getInet4Address(colName); - } - - @Override - public Inet6Address getInet6Address(String colName) { - return reader.getInet6Address(colName); - } - - @Override - public UUID getUUID(String colName) { - return reader.getUUID(colName); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(String colName) { - return reader.getGeoPoint(colName); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(String colName) { - return reader.getGeoRing(colName); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { - return reader.getGeoPolygon(colName); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { - return reader.getGeoMultiPolygon(colName); - } - - @Override - public List getList(String colName) { - return reader.getList(colName); - } - - @Override - public byte[] getByteArray(String colName) { - return reader.getByteArray(colName); - } - - @Override - public int[] getIntArray(String colName) { - return reader.getIntArray(colName); - } - - @Override - public long[] getLongArray(String colName) { - return reader.getLongArray(colName); - } - - @Override - public float[] getFloatArray(String colName) { - return reader.getFloatArray(colName); - } - - @Override - public double[] getDoubleArray(String colName) { - return reader.getDoubleArray(colName); - } - - @Override - public boolean[] getBooleanArray(String colName) { - return reader.getBooleanArray(colName); - } - - @Override - public short[] getShortArray(String colName) { - return reader.getShortArray(colName); - } - - @Override - public String[] getStringArray(String colName) { - return reader.getStringArray(colName); - } - - @Override - public Object[] getObjectArray(String colName) { - return reader.getObjectArray(colName); - } - - @Override - public String getString(int index) { - return reader.getString(index); - } - - @Override - public boolean hasValue(int colIndex) { - return reader.hasValue(colIndex); - } - - @Override - public boolean hasValue(String colName) { - return reader.hasValue(colName); - } - - @Override - public byte getByte(int index) { - return reader.getByte(index); - } - - @Override - public short getShort(int index) { - return reader.getShort(index); - } - - @Override - public int getInteger(int index) { - return reader.getInteger(index); - } - - @Override - public long getLong(int index) { - return reader.getLong(index); - } - - @Override - public float getFloat(int index) { - return reader.getFloat(index); - } - - @Override - public double getDouble(int index) { - return reader.getDouble(index); - } - - @Override - public boolean getBoolean(int index) { - return reader.getBoolean(index); - } - - @Override - public BigInteger getBigInteger(int index) { - return reader.getBigInteger(index); - } - - @Override - public BigDecimal getBigDecimal(int index) { - return reader.getBigDecimal(index); - } - - @Override - public Instant getInstant(int index) { - return reader.getInstant(index); - } - - @Override - public ZonedDateTime getZonedDateTime(int index) { - return reader.getZonedDateTime(index); - } - - @Override - public Duration getDuration(int index) { - return reader.getDuration(index); - } - - @Override - public TemporalAmount getTemporalAmount(int index) { - return reader.getTemporalAmount(index); - } - - @Override - public Inet4Address getInet4Address(int index) { - return reader.getInet4Address(index); - } - - @Override - public Inet6Address getInet6Address(int index) { - return reader.getInet6Address(index); - } - - @Override - public UUID getUUID(int index) { - return reader.getUUID(index); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(int index) { - return reader.getGeoPoint(index); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(int index) { - return reader.getGeoRing(index); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - return reader.getGeoPolygon(index); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - return reader.getGeoMultiPolygon(index); - } - - @Override - public List getList(int index) { - return reader.getList(index); - } - - @Override - public byte[] getByteArray(int index) { - return reader.getByteArray(index); - } - - @Override - public int[] getIntArray(int index) { - return reader.getIntArray(index); - } - - @Override - public long[] getLongArray(int index) { - return reader.getLongArray(index); - } - - @Override - public float[] getFloatArray(int index) { - return reader.getFloatArray(index); - } - - @Override - public double[] getDoubleArray(int index) { - return reader.getDoubleArray(index); - } - - @Override - public boolean[] getBooleanArray(int index) { - return reader.getBooleanArray(index); - } - - @Override - public short[] getShortArray(int index) { - return reader.getShortArray(index); - } - - @Override - public String[] getStringArray(int index) { - return reader.getStringArray(index); - } - - @Override - public Object[] getObjectArray(int index) { - return reader.getObjectArray(index); - } - - @Override - public Object[] getTuple(int index) { - return reader.getTuple(index); - } - - @Override - public Object[] getTuple(String colName) { - return reader.getTuple(colName); - } - - @Override - public byte getEnum8(String colName) { - return reader.getEnum8(colName); - } - - @Override - public byte getEnum8(int index) { - return reader.getEnum8(index); - } - - @Override - public short getEnum16(String colName) { - return reader.getEnum16(colName); - } - - @Override - public short getEnum16(int index) { - return reader.getEnum16(index); - } - - @Override - public LocalDate getLocalDate(String colName) { - return reader.getLocalDate(colName); - } - - @Override - public LocalDate getLocalDate(int index) { - return reader.getLocalDate(index); - } - - @Override - public LocalTime getLocalTime(String colName) { - return reader.getLocalTime(colName); - } - - @Override - public LocalTime getLocalTime(int index) { - return reader.getLocalTime(index); - } - - @Override - public LocalDateTime getLocalDateTime(String colName) { - return reader.getLocalDateTime(colName); - } - - @Override - public LocalDateTime getLocalDateTime(int index) { - return reader.getLocalDateTime(index); - } - - @Override - public OffsetDateTime getOffsetDateTime(String colName) { - return reader.getOffsetDateTime(colName); - } - - @Override - public OffsetDateTime getOffsetDateTime(int index) { - return reader.getOffsetDateTime(index); - } - - @Override - public Object getObject(String colName) { - return reader.readValue(colName); - } - - @Override - public Object getObject(int index) { - return reader.readValue(index); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(String colName) { - return reader.readValue(colName); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(int index) { - return reader.readValue(index); - } - - @Override - public TableSchema getSchema() { - return reader.getSchema(); - } - - @Override - public Map getValues() { - return this.getSchema().getColumns().stream().collect(Collectors.toMap( - ClickHouseColumn::getColumnName, - column -> this.getObject(column.getColumnName()))); - } -} +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.value.*; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.time.*; +import java.time.temporal.TemporalAmount; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +public class BinaryReaderBackedRecord implements GenericRecord { + + private final ClickHouseBinaryFormatReader reader; + + public BinaryReaderBackedRecord(ClickHouseBinaryFormatReader reader) { + this.reader = reader; + } + + @Override + public String getString(String colName) { + return reader.getString(colName); + } + + @Override + public byte getByte(String colName) { + return reader.getByte(colName); + } + + @Override + public short getShort(String colName) { + return reader.getShort(colName); + } + + @Override + public int getInteger(String colName) { + return reader.getInteger(colName); + } + + @Override + public long getLong(String colName) { + return reader.getLong(colName); + } + + @Override + public float getFloat(String colName) { + return reader.getFloat(colName); + } + + @Override + public double getDouble(String colName) { + return reader.getDouble(colName); + } + + @Override + public boolean getBoolean(String colName) { + return reader.getBoolean(colName); + } + + @Override + public BigInteger getBigInteger(String colName) { + return reader.getBigInteger(colName); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + return reader.getBigDecimal(colName); + } + + @Override + public Instant getInstant(String colName) { + return reader.getInstant(colName); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + return reader.getZonedDateTime(colName); + } + + @Override + public Duration getDuration(String colName) { + return reader.getDuration(colName); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return reader.getTemporalAmount(colName); + } + + @Override + public Inet4Address getInet4Address(String colName) { + return reader.getInet4Address(colName); + } + + @Override + public Inet6Address getInet6Address(String colName) { + return reader.getInet6Address(colName); + } + + @Override + public UUID getUUID(String colName) { + return reader.getUUID(colName); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + return reader.getGeoPoint(colName); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + return reader.getGeoRing(colName); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + return reader.getGeoPolygon(colName); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + return reader.getGeoMultiPolygon(colName); + } + + @Override + public List getList(String colName) { + return reader.getList(colName); + } + + @Override + public byte[] getByteArray(String colName) { + return reader.getByteArray(colName); + } + + @Override + public int[] getIntArray(String colName) { + return reader.getIntArray(colName); + } + + @Override + public long[] getLongArray(String colName) { + return reader.getLongArray(colName); + } + + @Override + public float[] getFloatArray(String colName) { + return reader.getFloatArray(colName); + } + + @Override + public double[] getDoubleArray(String colName) { + return reader.getDoubleArray(colName); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return reader.getBooleanArray(colName); + } + + @Override + public short[] getShortArray(String colName) { + return reader.getShortArray(colName); + } + + @Override + public String[] getStringArray(String colName) { + return reader.getStringArray(colName); + } + + @Override + public Object[] getObjectArray(String colName) { + return reader.getObjectArray(colName); + } + + @Override + public String getString(int index) { + return reader.getString(index); + } + + @Override + public boolean hasValue(int colIndex) { + return reader.hasValue(colIndex); + } + + @Override + public boolean hasValue(String colName) { + return reader.hasValue(colName); + } + + @Override + public byte getByte(int index) { + return reader.getByte(index); + } + + @Override + public short getShort(int index) { + return reader.getShort(index); + } + + @Override + public int getInteger(int index) { + return reader.getInteger(index); + } + + @Override + public long getLong(int index) { + return reader.getLong(index); + } + + @Override + public float getFloat(int index) { + return reader.getFloat(index); + } + + @Override + public double getDouble(int index) { + return reader.getDouble(index); + } + + @Override + public boolean getBoolean(int index) { + return reader.getBoolean(index); + } + + @Override + public BigInteger getBigInteger(int index) { + return reader.getBigInteger(index); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return reader.getBigDecimal(index); + } + + @Override + public Instant getInstant(int index) { + return reader.getInstant(index); + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + return reader.getZonedDateTime(index); + } + + @Override + public Duration getDuration(int index) { + return reader.getDuration(index); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return reader.getTemporalAmount(index); + } + + @Override + public Inet4Address getInet4Address(int index) { + return reader.getInet4Address(index); + } + + @Override + public Inet6Address getInet6Address(int index) { + return reader.getInet6Address(index); + } + + @Override + public UUID getUUID(int index) { + return reader.getUUID(index); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + return reader.getGeoPoint(index); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + return reader.getGeoRing(index); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + return reader.getGeoPolygon(index); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + return reader.getGeoMultiPolygon(index); + } + + @Override + public List getList(int index) { + return reader.getList(index); + } + + @Override + public byte[] getByteArray(int index) { + return reader.getByteArray(index); + } + + @Override + public int[] getIntArray(int index) { + return reader.getIntArray(index); + } + + @Override + public long[] getLongArray(int index) { + return reader.getLongArray(index); + } + + @Override + public float[] getFloatArray(int index) { + return reader.getFloatArray(index); + } + + @Override + public double[] getDoubleArray(int index) { + return reader.getDoubleArray(index); + } + + @Override + public boolean[] getBooleanArray(int index) { + return reader.getBooleanArray(index); + } + + @Override + public short[] getShortArray(int index) { + return reader.getShortArray(index); + } + + @Override + public String[] getStringArray(int index) { + return reader.getStringArray(index); + } + + @Override + public Object[] getObjectArray(int index) { + return reader.getObjectArray(index); + } + + @Override + public Object[] getTuple(int index) { + return reader.getTuple(index); + } + + @Override + public Object[] getTuple(String colName) { + return reader.getTuple(colName); + } + + @Override + public byte getEnum8(String colName) { + return reader.getEnum8(colName); + } + + @Override + public byte getEnum8(int index) { + return reader.getEnum8(index); + } + + @Override + public short getEnum16(String colName) { + return reader.getEnum16(colName); + } + + @Override + public short getEnum16(int index) { + return reader.getEnum16(index); + } + + @Override + public LocalDate getLocalDate(String colName) { + return reader.getLocalDate(colName); + } + + @Override + public LocalDate getLocalDate(int index) { + return reader.getLocalDate(index); + } + + @Override + public LocalTime getLocalTime(String colName) { + return reader.getLocalTime(colName); + } + + @Override + public LocalTime getLocalTime(int index) { + return reader.getLocalTime(index); + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + return reader.getLocalDateTime(colName); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + return reader.getLocalDateTime(index); + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + return reader.getOffsetDateTime(colName); + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + return reader.getOffsetDateTime(index); + } + + @Override + public Object getObject(String colName) { + return reader.readValue(colName); + } + + @Override + public Object getObject(int index) { + return reader.readValue(index); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + return reader.readValue(colName); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + return reader.readValue(index); + } + + @Override + public TableSchema getSchema() { + return reader.getSchema(); + } + + @Override + public Map getValues() { + return this.getSchema().getColumns().stream().collect(Collectors.toMap( + ClickHouseColumn::getColumnName, + column -> this.getObject(column.getColumnName()))); + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index d83dd22c0..07e65de39 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -1,686 +1,686 @@ -package com.clickhouse.client.api.data_formats.internal; - -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.internal.DataTypeConverter; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.client.api.query.NullValueException; -import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.value.ClickHouseBitmap; -import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoPointValue; -import com.clickhouse.data.value.ClickHouseGeoPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoRingValue; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.time.temporal.TemporalAmount; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Function; - -public class MapBackedRecord implements GenericRecord { - - private final Map record; - - private final TableSchema schema; - - private Map[] columnConverters; - - private DataTypeConverter dataTypeConverter; - - public MapBackedRecord(Map record, Map[] columnConverters, TableSchema schema) { - this.record = new HashMap<>(record); - this.schema = schema; - this.columnConverters = columnConverters; - this.dataTypeConverter = DataTypeConverter.INSTANCE; - } - - public T readValue(int colIndex) { - if (colIndex < 1 || colIndex > schema.getColumns().size()) { - throw new ClientException("Column index out of bounds: " + colIndex); - } - - return (T) record.get(schema.columnIndexToName(colIndex)); - } - - public T readValue(String colName) { - return (T) record.get(colName); - } - - @Override - public String getString(String colName) { - return dataTypeConverter.convertToString(readValue(colName), schema.getColumnByName(colName)); - } - - @Override - public String getString(int index) { - return getString(schema.columnIndexToName(index)); - } - - private T readNumberValue(String colName, NumberConverter.NumberType targetType) { - int colIndex = schema.nameToIndex(colName); - - Function converter = (Function) columnConverters[colIndex].get(targetType); - if (converter != null) { - Object value = readValue(colName); - if (value == null) { - if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { - return null; - } - throw new NullValueException("Column " + colName + " has null value and it cannot be cast to " + - targetType.getTypeName()); - } - return (T) converter.apply(value); - } else { - String columnTypeName = schema.getColumnByName(colName).getDataType().name(); - throw new ClientException("Column '" + colName + "' of type " + columnTypeName + - " cannot be converted to '" + targetType.getTypeName() + "' value"); - } - } - - @Override - public byte getByte(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Byte); - } - - @Override - public short getShort(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Short); - } - - @Override - public int getInteger(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Int); - } - - @Override - public long getLong(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Long); - } - - @Override - public float getFloat(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Float); - } - - @Override - public double getDouble(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Double); - } - - @Override - public boolean getBoolean(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.Boolean); - } - - @Override - public BigInteger getBigInteger(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.BigInteger); - } - - @Override - public BigDecimal getBigDecimal(String colName) { - return readNumberValue(colName, NumberConverter.NumberType.BigDecimal); - } - - @Override - public Instant getInstant(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - int colIndex = column.getColumnIndex(); - switch (column.getValueDataType()) { - case Date: - case Date32: - LocalDate date = getLocalDate(colIndex); - return date == null ? null : Instant.from(date); - case Time: - case Time64: - LocalDateTime time = getLocalDateTime(colName); - return time == null ? null : time.toInstant(ZoneOffset.UTC); - case DateTime: - case DateTime64: - case DateTime32: - ZonedDateTime zdt = getZonedDateTime(colName); - return zdt == null ? null : zdt.toInstant(); - case Dynamic: - case Variant: - Object value = readValue(colName); - Instant instant = AbstractBinaryFormatReader.objectToInstant(value); - if (value == null || instant != null) { - return instant; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); - } - - @Override - public ZonedDateTime getZonedDateTime(String colName) { - return getZonedDateTime(schema.nameToColumnIndex(colName)); - } - - @Override - public Duration getDuration(String colName) { - TemporalAmount temporalAmount = readValue(colName); - return temporalAmount == null ? null : Duration.from(temporalAmount); - } - - @Override - public TemporalAmount getTemporalAmount(String colName) { - return readValue(colName); - } - - @Override - public Inet4Address getInet4Address(String colName) { - Object val = readValue(colName); - return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); - } - - @Override - public Inet6Address getInet6Address(String colName) { - Object val = readValue(colName); - return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); - } - - @Override - public UUID getUUID(String colName) { - return readValue(colName); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { - Object val = readValue(colName); - return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); - } - - - @Override - public List getList(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).asList(); - } else if (value instanceof List) { - return (List) value; - } else { - throw new ClientException("Column is not of array type"); - } - } - - - private T getPrimitiveArray(String colName) { - BinaryStreamReader.ArrayValue array = readValue(colName); - if (array == null) { - return null; - } - if (array.itemType.isPrimitive()) { - return (T) array.array; - } else { - throw new ClientException("Array is not of primitive type"); - } - } - - @Override - public byte[] getByteArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public int[] getIntArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public long[] getLongArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public float[] getFloatArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public double[] getDoubleArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public boolean[] getBooleanArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public short[] getShortArray(String colName) { - return getPrimitiveArray(colName); - } - - @Override - public String[] getStringArray(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; - if (array.itemType == String.class) { - return (String[]) array.getArray(); - } else if (array.itemType == BinaryStreamReader.EnumValue.class) { - BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); - return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); - } else { - throw new ClientException("Not an array of strings"); - } - } - throw new ClientException("Column is not of array type"); - } - - @Override - public boolean hasValue(int colIndex) { - return hasValue(schema.columnIndexToName(colIndex)); - } - - @Override - public boolean hasValue(String colName) { - return record.containsKey(colName); - } - - @Override - public byte getByte(int index) { - return getByte(schema.columnIndexToName(index)); - } - - @Override - public short getShort(int index) { - return getShort(schema.columnIndexToName(index)); - } - - @Override - public int getInteger(int index) { - return getInteger(schema.columnIndexToName(index)); - } - - @Override - public long getLong(int index) { - return getLong(schema.columnIndexToName(index)); - } - - @Override - public float getFloat(int index) { - return getFloat(schema.columnIndexToName(index)); - } - - @Override - public double getDouble(int index) { - return getDouble(schema.columnIndexToName(index)); - } - - @Override - public boolean getBoolean(int index) { - return getBoolean(schema.columnIndexToName(index)); - } - - @Override - public BigInteger getBigInteger(int index) { - return getBigInteger(schema.columnIndexToName(index)); - } - - @Override - public BigDecimal getBigDecimal(int index) { - return getBigDecimal(schema.columnIndexToName(index)); - } - - @Override - public Instant getInstant(int index) { - return getInstant(schema.columnIndexToName(index)); - } - - @Override - public ZonedDateTime getZonedDateTime(int index) { - ClickHouseColumn column = schema.getColumnByIndex(index); - switch (column.getValueDataType()) { - case DateTime: - case DateTime64: - case DateTime32: - return readValue(index); - case Dynamic: - case Variant: - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof ZonedDateTime) { - return (ZonedDateTime) value; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); - } - - @Override - public Duration getDuration(int index) { - return getDuration(schema.columnIndexToName(index)); - } - - @Override - public TemporalAmount getTemporalAmount(int index) { - return readValue(index); - } - - @Override - public Inet4Address getInet4Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); - } - - @Override - public Inet6Address getInet6Address(int index) { - Object val = readValue(index); - return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); - } - - @Override - public UUID getUUID(int index) { - return readValue(index); - } - - @Override - public ClickHouseGeoPointValue getGeoPoint(int index) { - return getGeoPoint(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseGeoRingValue getGeoRing(int index) { - return getGeoRing(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - return getGeoPolygon(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - return getGeoMultiPolygon(schema.columnIndexToName(index)); - } - - @Override - public List getList(int index) { - return getList(schema.columnIndexToName(index)); - } - - @Override - public byte[] getByteArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public int[] getIntArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public long[] getLongArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public float[] getFloatArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public double[] getDoubleArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public boolean[] getBooleanArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public short[] getShortArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); - } - - @Override - public String[] getStringArray(int index) { - return getStringArray(schema.columnIndexToName(index)); - } - - @Override - public Object[] getObjectArray(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } - if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); - } else if (value instanceof List) { - return ((List) value).toArray(new Object[0]); - } - throw new ClientException("Column is not of array type"); - } - - @Override - public Object[] getObjectArray(int index) { - return getObjectArray(schema.columnIndexToName(index)); - } - - @Override - public Object[] getTuple(int index) { - return readValue(index); - } - - @Override - public Object[] getTuple(String colName) { - return readValue(colName); - } - - @Override - public byte getEnum8(String colName) { - Object val = readValue(colName); - if (val == null) { - throw new NullValueException("Column " + colName + " has null value and it cannot be cast to byte"); - } - if (val instanceof BinaryStreamReader.EnumValue) { - return ((BinaryStreamReader.EnumValue) val).byteValue(); - } - return (byte) val; - } - - @Override - public byte getEnum8(int index) { - return getEnum8(schema.columnIndexToName(index)); - } - - @Override - public short getEnum16(String colName) { - Object val = readValue(colName); - if (val == null) { - throw new NullValueException("Column " + colName + " has null value and it cannot be cast to short"); - } - if (val instanceof BinaryStreamReader.EnumValue) { - return ((BinaryStreamReader.EnumValue) val).shortValue(); - } - return (short) val; - } - - @Override - public short getEnum16(int index) { - return getEnum16(schema.columnIndexToName(index)); - } - - @Override - public LocalDate getLocalDate(int index) { - return getLocalDate(schema.columnIndexToName(index)); - } - - @Override - public LocalDate getLocalDate(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case Date: - case Date32: - return (LocalDate) getObject(colName); - case DateTime: - case DateTime32: - case DateTime64: - LocalDateTime dt = getLocalDateTime(colName); - return dt == null ? null : dt.toLocalDate(); - case Dynamic: - case Variant: - Object value = getObject(colName); - LocalDate localDate = AbstractBinaryFormatReader.objectToLocalDate(value); - if (value == null || localDate != null) { - return localDate; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); - } - - @Override - public LocalTime getLocalTime(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case Time: - case Time64: - LocalDateTime val = (LocalDateTime) getObject(colName); - return val == null ? null : val.toLocalTime(); - case DateTime: - case DateTime32: - case DateTime64: - LocalDateTime dt = getLocalDateTime(colName); - return dt == null ? null : dt.toLocalTime(); - case Dynamic: - case Variant: - Object value = getObject(colName); - LocalTime localTime = AbstractBinaryFormatReader.objectToLocalTime(value); - if (value == null || localTime != null) { - return localTime; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); - } - - @Override - public LocalTime getLocalTime(int index) { - return getLocalTime(schema.columnIndexToName(index)); - } - - @Override - public LocalDateTime getLocalDateTime(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case Time: - case Time64: - // Types present wide range of value so LocalDateTime let to access to actual value - return (LocalDateTime) getObject(colName); - case DateTime: - case DateTime32: - case DateTime64: - ZonedDateTime val = (ZonedDateTime) readValue(colName); - return val == null ? null : val.toLocalDateTime(); - case Dynamic: - case Variant: - Object value = getObject(colName); - LocalDateTime localDateTime = AbstractBinaryFormatReader.objectToLocalDateTime(value); - if (value == null || localDateTime != null) { - return localDateTime; - } - break; - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); - } - - @Override - public LocalDateTime getLocalDateTime(int index) { - return getLocalDateTime(schema.columnIndexToName(index)); - } - - @Override - public OffsetDateTime getOffsetDateTime(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - switch(column.getValueDataType()) { - case DateTime: - case DateTime32: - case DateTime64: - case Dynamic: - case Variant: - ZonedDateTime val = getZonedDateTime(colName); - return val == null ? null : val.toOffsetDateTime(); - default: - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDataTime"); - } - } - - @Override - public OffsetDateTime getOffsetDateTime(int index) { - return getOffsetDateTime(schema.columnIndexToName(index)); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(String colName) { - return readValue(colName); - } - - @Override - public ClickHouseBitmap getClickHouseBitmap(int index) { - return readValue(index); - } - - @Override - public TableSchema getSchema() { - return this.schema; - } - - @Override - public Object getObject(String colName) { - return readValue(colName); - } - - @Override - public Object getObject(int index) { - return readValue(index); - } - - @Override - public Map getValues() { - return this.record; - } -} +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.internal.DataTypeConverter; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.NullValueException; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +public class MapBackedRecord implements GenericRecord { + + private final Map record; + + private final TableSchema schema; + + private Map[] columnConverters; + + private DataTypeConverter dataTypeConverter; + + public MapBackedRecord(Map record, Map[] columnConverters, TableSchema schema) { + this.record = new HashMap<>(record); + this.schema = schema; + this.columnConverters = columnConverters; + this.dataTypeConverter = DataTypeConverter.INSTANCE; + } + + public T readValue(int colIndex) { + if (colIndex < 1 || colIndex > schema.getColumns().size()) { + throw new ClientException("Column index out of bounds: " + colIndex); + } + + return (T) record.get(schema.columnIndexToName(colIndex)); + } + + public T readValue(String colName) { + return (T) record.get(colName); + } + + @Override + public String getString(String colName) { + return dataTypeConverter.convertToString(readValue(colName), schema.getColumnByName(colName)); + } + + @Override + public String getString(int index) { + return getString(schema.columnIndexToName(index)); + } + + private T readNumberValue(String colName, NumberConverter.NumberType targetType) { + int colIndex = schema.nameToIndex(colName); + + Function converter = (Function) columnConverters[colIndex].get(targetType); + if (converter != null) { + Object value = readValue(colName); + if (value == null) { + if (targetType == NumberConverter.NumberType.BigInteger || targetType == NumberConverter.NumberType.BigDecimal) { + return null; + } + throw new NullValueException("Column " + colName + " has null value and it cannot be cast to " + + targetType.getTypeName()); + } + return (T) converter.apply(value); + } else { + String columnTypeName = schema.getColumnByName(colName).getDataType().name(); + throw new ClientException("Column '" + colName + "' of type " + columnTypeName + + " cannot be converted to '" + targetType.getTypeName() + "' value"); + } + } + + @Override + public byte getByte(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Byte); + } + + @Override + public short getShort(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Short); + } + + @Override + public int getInteger(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Int); + } + + @Override + public long getLong(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Long); + } + + @Override + public float getFloat(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Float); + } + + @Override + public double getDouble(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Double); + } + + @Override + public boolean getBoolean(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.Boolean); + } + + @Override + public BigInteger getBigInteger(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.BigInteger); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + return readNumberValue(colName, NumberConverter.NumberType.BigDecimal); + } + + @Override + public Instant getInstant(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + int colIndex = column.getColumnIndex(); + switch (column.getValueDataType()) { + case Date: + case Date32: + LocalDate date = getLocalDate(colIndex); + return date == null ? null : Instant.from(date); + case Time: + case Time64: + LocalDateTime time = getLocalDateTime(colName); + return time == null ? null : time.toInstant(ZoneOffset.UTC); + case DateTime: + case DateTime64: + case DateTime32: + ZonedDateTime zdt = getZonedDateTime(colName); + return zdt == null ? null : zdt.toInstant(); + case Dynamic: + case Variant: + Object value = readValue(colName); + Instant instant = AbstractBinaryFormatReader.objectToInstant(value); + if (value == null || instant != null) { + return instant; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + return getZonedDateTime(schema.nameToColumnIndex(colName)); + } + + @Override + public Duration getDuration(String colName) { + TemporalAmount temporalAmount = readValue(colName); + return temporalAmount == null ? null : Duration.from(temporalAmount); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return readValue(colName); + } + + @Override + public Inet4Address getInet4Address(String colName) { + Object val = readValue(colName); + return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); + } + + @Override + public Inet6Address getInet6Address(String colName) { + Object val = readValue(colName); + return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); + } + + @Override + public UUID getUUID(String colName) { + return readValue(colName); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoPointValue.of((double[]) val); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoRingValue.of((double[][]) val); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoPolygonValue.of((double[][][]) val); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + Object val = readValue(colName); + return val == null ? null : ClickHouseGeoMultiPolygonValue.of((double[][][][]) val); + } + + + @Override + public List getList(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).asList(); + } else if (value instanceof List) { + return (List) value; + } else { + throw new ClientException("Column is not of array type"); + } + } + + + private T getPrimitiveArray(String colName) { + BinaryStreamReader.ArrayValue array = readValue(colName); + if (array == null) { + return null; + } + if (array.itemType.isPrimitive()) { + return (T) array.array; + } else { + throw new ClientException("Array is not of primitive type"); + } + } + + @Override + public byte[] getByteArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public int[] getIntArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public long[] getLongArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public float[] getFloatArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public double[] getDoubleArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public short[] getShortArray(String colName) { + return getPrimitiveArray(colName); + } + + @Override + public String[] getStringArray(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType == String.class) { + return (String[]) array.getArray(); + } else if (array.itemType == BinaryStreamReader.EnumValue.class) { + BinaryStreamReader.EnumValue[] enumValues = (BinaryStreamReader.EnumValue[]) array.getArray(); + return Arrays.stream(enumValues).map(BinaryStreamReader.EnumValue::getName).toArray(String[]::new); + } else { + throw new ClientException("Not an array of strings"); + } + } + throw new ClientException("Column is not of array type"); + } + + @Override + public boolean hasValue(int colIndex) { + return hasValue(schema.columnIndexToName(colIndex)); + } + + @Override + public boolean hasValue(String colName) { + return record.containsKey(colName); + } + + @Override + public byte getByte(int index) { + return getByte(schema.columnIndexToName(index)); + } + + @Override + public short getShort(int index) { + return getShort(schema.columnIndexToName(index)); + } + + @Override + public int getInteger(int index) { + return getInteger(schema.columnIndexToName(index)); + } + + @Override + public long getLong(int index) { + return getLong(schema.columnIndexToName(index)); + } + + @Override + public float getFloat(int index) { + return getFloat(schema.columnIndexToName(index)); + } + + @Override + public double getDouble(int index) { + return getDouble(schema.columnIndexToName(index)); + } + + @Override + public boolean getBoolean(int index) { + return getBoolean(schema.columnIndexToName(index)); + } + + @Override + public BigInteger getBigInteger(int index) { + return getBigInteger(schema.columnIndexToName(index)); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return getBigDecimal(schema.columnIndexToName(index)); + } + + @Override + public Instant getInstant(int index) { + return getInstant(schema.columnIndexToName(index)); + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + ClickHouseColumn column = schema.getColumnByIndex(index); + switch (column.getValueDataType()) { + case DateTime: + case DateTime64: + case DateTime32: + return readValue(index); + case Dynamic: + case Variant: + Object value = readValue(index); + if (value == null) { + return null; + } else if (value instanceof ZonedDateTime) { + return (ZonedDateTime) value; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime"); + } + + @Override + public Duration getDuration(int index) { + return getDuration(schema.columnIndexToName(index)); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return readValue(index); + } + + @Override + public Inet4Address getInet4Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv4((java.net.InetAddress) val); + } + + @Override + public Inet6Address getInet6Address(int index) { + Object val = readValue(index); + return val == null ? null : InetAddressConverter.convertToIpv6((java.net.InetAddress) val); + } + + @Override + public UUID getUUID(int index) { + return readValue(index); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + return getGeoPoint(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + return getGeoRing(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + return getGeoPolygon(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + return getGeoMultiPolygon(schema.columnIndexToName(index)); + } + + @Override + public List getList(int index) { + return getList(schema.columnIndexToName(index)); + } + + @Override + public byte[] getByteArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public int[] getIntArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public long[] getLongArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public float[] getFloatArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public double[] getDoubleArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public boolean[] getBooleanArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public short[] getShortArray(int index) { + return getPrimitiveArray(schema.columnIndexToName(index)); + } + + @Override + public String[] getStringArray(int index) { + return getStringArray(schema.columnIndexToName(index)); + } + + @Override + public Object[] getObjectArray(String colName) { + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).toObjectArray(); + } else if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + throw new ClientException("Column is not of array type"); + } + + @Override + public Object[] getObjectArray(int index) { + return getObjectArray(schema.columnIndexToName(index)); + } + + @Override + public Object[] getTuple(int index) { + return readValue(index); + } + + @Override + public Object[] getTuple(String colName) { + return readValue(colName); + } + + @Override + public byte getEnum8(String colName) { + Object val = readValue(colName); + if (val == null) { + throw new NullValueException("Column " + colName + " has null value and it cannot be cast to byte"); + } + if (val instanceof BinaryStreamReader.EnumValue) { + return ((BinaryStreamReader.EnumValue) val).byteValue(); + } + return (byte) val; + } + + @Override + public byte getEnum8(int index) { + return getEnum8(schema.columnIndexToName(index)); + } + + @Override + public short getEnum16(String colName) { + Object val = readValue(colName); + if (val == null) { + throw new NullValueException("Column " + colName + " has null value and it cannot be cast to short"); + } + if (val instanceof BinaryStreamReader.EnumValue) { + return ((BinaryStreamReader.EnumValue) val).shortValue(); + } + return (short) val; + } + + @Override + public short getEnum16(int index) { + return getEnum16(schema.columnIndexToName(index)); + } + + @Override + public LocalDate getLocalDate(int index) { + return getLocalDate(schema.columnIndexToName(index)); + } + + @Override + public LocalDate getLocalDate(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case Date: + case Date32: + return (LocalDate) getObject(colName); + case DateTime: + case DateTime32: + case DateTime64: + LocalDateTime dt = getLocalDateTime(colName); + return dt == null ? null : dt.toLocalDate(); + case Dynamic: + case Variant: + Object value = getObject(colName); + LocalDate localDate = AbstractBinaryFormatReader.objectToLocalDate(value); + if (value == null || localDate != null) { + return localDate; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDate"); + } + + @Override + public LocalTime getLocalTime(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case Time: + case Time64: + LocalDateTime val = (LocalDateTime) getObject(colName); + return val == null ? null : val.toLocalTime(); + case DateTime: + case DateTime32: + case DateTime64: + LocalDateTime dt = getLocalDateTime(colName); + return dt == null ? null : dt.toLocalTime(); + case Dynamic: + case Variant: + Object value = getObject(colName); + LocalTime localTime = AbstractBinaryFormatReader.objectToLocalTime(value); + if (value == null || localTime != null) { + return localTime; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalTime"); + } + + @Override + public LocalTime getLocalTime(int index) { + return getLocalTime(schema.columnIndexToName(index)); + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case Time: + case Time64: + // Types present wide range of value so LocalDateTime let to access to actual value + return (LocalDateTime) getObject(colName); + case DateTime: + case DateTime32: + case DateTime64: + ZonedDateTime val = (ZonedDateTime) readValue(colName); + return val == null ? null : val.toLocalDateTime(); + case Dynamic: + case Variant: + Object value = getObject(colName); + LocalDateTime localDateTime = AbstractBinaryFormatReader.objectToLocalDateTime(value); + if (value == null || localDateTime != null) { + return localDateTime; + } + break; + } + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to LocalDateTime"); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + return getLocalDateTime(schema.columnIndexToName(index)); + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + ClickHouseColumn column = schema.getColumnByName(colName); + switch(column.getValueDataType()) { + case DateTime: + case DateTime32: + case DateTime64: + case Dynamic: + case Variant: + ZonedDateTime val = getZonedDateTime(colName); + return val == null ? null : val.toOffsetDateTime(); + default: + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to OffsetDataTime"); + } + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + return getOffsetDateTime(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + return readValue(colName); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + return readValue(index); + } + + @Override + public TableSchema getSchema() { + return this.schema; + } + + @Override + public Object getObject(String colName) { + return readValue(colName); + } + + @Override + public Object getObject(int index) { + return readValue(index); + } + + @Override + public Map getValues() { + return this.record; + } +}