From 1ee866823f0610e7d6141db457563304687bbd4a Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Mon, 29 Nov 2021 21:20:33 +0800 Subject: [PATCH] Introduce ClickHouseInputStream for tiny reads --- .../clickhouse/client/ClickHouseChecker.java | 21 + .../client/ClickHouseDataProcessor.java | 5 +- .../client/ClickHouseDataStreamFactory.java | 5 +- .../client/ClickHouseDeserializer.java | 4 +- .../client/ClickHouseInputStream.java | 95 +++ .../clickhouse/client/ClickHouseValues.java | 281 +++---- .../client/data/BinaryStreamUtils.java | 178 ++--- .../data/ClickHouseBigDecimalValue.java | 13 +- .../client/data/ClickHousePipedStream.java | 31 +- .../data/ClickHouseRowBinaryProcessor.java | 8 +- .../client/data/ClickHouseStreamResponse.java | 25 +- .../client/data/ClickHouseStringValue.java | 5 + .../data/ClickHouseTabSeparatedProcessor.java | 22 +- .../client/data/BinaryStreamUtilsTest.java | 749 ++++++++++++------ .../client/grpc/ClickHouseGrpcResponse.java | 6 +- .../client/http/ClickHouseHttpResponse.java | 25 +- .../clickhouse/jdbc/ClickHouseConnection.java | 34 +- .../clickhouse/jdbc/ClickHouseResultSet.java | 38 +- .../internal/ClickHouseConnectionImpl.java | 52 +- .../jdbc/internal/FakeTransaction.java | 5 + .../internal/SqlBasedPreparedStatement.java | 14 +- .../StreamBasedPreparedStatement.java | 17 +- .../jdbc/CombinedResultSetTest.java | 55 +- 23 files changed, 1109 insertions(+), 579 deletions(-) create mode 100644 clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java index 2fe231a24..945d0592a 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java @@ -51,6 +51,27 @@ public static long between(long value, long minValue, long maxValue) { return between(value, DEFAULT_NAME, minValue, maxValue); } + /** + * Checks if the given {@code value} is between {@code minValue} and + * {@code maxValue} inclusive and throws a customized + * {@link IllegalArgumentException} if it is NOT. + * + * @param value the value to check + * @param name name of the value + * @param minValue minimum value to compare with + * @param maxValue maximum value to compare with + * @return the exact same value + * @throws IllegalArgumentException if the {@code value} is NOT between + * {@code minValue} and {@code maxValue} + */ + public static int between(byte value, String name, byte minValue, byte maxValue) { + if (value < minValue || value > maxValue) { + throw newException(ERR_SHOULD_BETWEEN, name, value, minValue, maxValue); + } + + return value; + } + /** * Checks if the given {@code value} is between {@code minValue} and * {@code maxValue} inclusive and throws a customized diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java index 357f8c1d4..9c33e5e1a 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java @@ -1,7 +1,6 @@ package com.clickhouse.client; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; import java.util.Collections; @@ -34,7 +33,7 @@ protected static void buildMappings( } protected final ClickHouseConfig config; - protected final InputStream input; + protected final ClickHouseInputStream input; protected final OutputStream output; protected final List columns; protected final Map settings; @@ -60,7 +59,7 @@ protected static void buildMappings( * @param settings nullable settings * @throws IOException when failed to read columns from input stream */ - protected ClickHouseDataProcessor(ClickHouseConfig config, InputStream input, OutputStream output, + protected ClickHouseDataProcessor(ClickHouseConfig config, ClickHouseInputStream input, OutputStream output, List columns, Map settings) throws IOException { this.config = ClickHouseChecker.nonNull(config, "config"); if (input == null && output == null) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java index c9744ea3c..64198dfaf 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java @@ -1,7 +1,6 @@ package com.clickhouse.client; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.List; import java.util.Map; @@ -40,8 +39,8 @@ public static ClickHouseDataStreamFactory getInstance() { * @return data processor * @throws IOException when failed to read columns from input stream */ - public ClickHouseDataProcessor getProcessor(ClickHouseConfig config, InputStream input, OutputStream output, - Map settings, List columns) throws IOException { + public ClickHouseDataProcessor getProcessor(ClickHouseConfig config, ClickHouseInputStream input, + OutputStream output, Map settings, List columns) throws IOException { ClickHouseFormat format = ClickHouseChecker.nonNull(config, "config").getFormat(); ClickHouseDataProcessor processor; if (ClickHouseFormat.RowBinary == format || ClickHouseFormat.RowBinaryWithNamesAndTypes == format) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDeserializer.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDeserializer.java index 8378a705c..abdca348d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDeserializer.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDeserializer.java @@ -1,7 +1,6 @@ package com.clickhouse.client; import java.io.IOException; -import java.io.InputStream; /** * Functional interface for deserialization. @@ -19,5 +18,6 @@ public interface ClickHouseDeserializer { * @return deserialized value which might be the same instance as {@code ref} * @throws IOException when failed to read data from input stream */ - T deserialize(T ref, ClickHouseConfig config, ClickHouseColumn column, InputStream input) throws IOException; + T deserialize(T ref, ClickHouseConfig config, ClickHouseColumn column, ClickHouseInputStream input) + throws IOException; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java new file mode 100644 index 000000000..1f4601b50 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -0,0 +1,95 @@ +package com.clickhouse.client; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + +/** + * Extended input stream. + */ +public abstract class ClickHouseInputStream extends InputStream { + static final class SimpleInputStream extends ClickHouseInputStream { + private final InputStream in; + + private boolean closed; + + protected SimpleInputStream(InputStream input) { + this.in = ClickHouseChecker.nonNull(input, "InputStream"); + this.closed = false; + } + + @Override + public int available() throws IOException { + return in.available(); + } + + @Override + public byte readByte() throws IOException { + int v = in.read(); + if (v == -1) { + close(); + throw new EOFException(); + } + + return (byte) v; + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public void close() throws IOException { + in.close(); + closed = true; + } + + @Override + public int read() throws IOException { + return in.read(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return in.read(b, off, len); + } + + @Override + public long skip(long n) throws IOException { + return in.skip(n); + } + } + + public static ClickHouseInputStream of(InputStream input) { + return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input : new SimpleInputStream(input); + } + + /** + * Reads an unsigned byte from the input stream. + * + * @return unsigned byte + * @throws IOException when failed to read value from input stream or reached + * end of the stream + */ + public int readUnsignedByte() throws IOException { + return 0xFF & readByte(); + } + + /** + * Reads one single byte from the input stream. This is faster than + * {@link #read()}. + * + * @return byte value if present + * @throws IOException when failed to read value from input stream or reached + * end of the stream + */ + public abstract byte readByte() throws IOException; + + /** + * Checks if the input stream has been closed or not. + * + * @return true if the input stream has been closed; false otherwise + */ + public abstract boolean isClosed(); +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java index 5a6472600..746302f84 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java @@ -823,7 +823,7 @@ public static Object createPrimitiveArray(Class clazz, int length, int level) } static ClickHouseArrayValue fillByteArray(ClickHouseArrayValue array, ClickHouseConfig config, - ClickHouseColumn column, InputStream input, ClickHouseDeserializer deserializer, + ClickHouseColumn column, ClickHouseInputStream input, ClickHouseDeserializer deserializer, int length) throws IOException { byte[] values = new byte[length]; ClickHouseValue ref = ClickHouseByteValue.ofNull(); @@ -909,149 +909,150 @@ public static ClickHouseValue newValue(ClickHouseDataType type) { private static ClickHouseValue newValue(ClickHouseDataType type, ClickHouseColumn column) { ClickHouseValue value = null; switch (type) { // still faster than EnumMap and with less overhead - case Enum: - case Enum8: - case Int8: - value = ClickHouseByteValue.ofNull(); - break; - case UInt8: - case Enum16: - case Int16: - value = ClickHouseShortValue.ofNull(); - break; - case UInt16: - case Int32: - value = ClickHouseIntegerValue.ofNull(); - break; - case UInt32: - case IntervalYear: - case IntervalQuarter: - case IntervalMonth: - case IntervalWeek: - case IntervalDay: - case IntervalHour: - case IntervalMinute: - case IntervalSecond: - case Int64: - value = ClickHouseLongValue.ofNull(false); - break; - case UInt64: - value = ClickHouseLongValue.ofNull(true); - break; - case Int128: - case UInt128: - case Int256: - case UInt256: - value = ClickHouseBigIntegerValue.ofNull(); - break; - case Float32: - value = ClickHouseFloatValue.ofNull(); - break; - case Float64: - value = ClickHouseDoubleValue.ofNull(); - break; - case Decimal: - case Decimal32: - case Decimal64: - case Decimal128: - case Decimal256: - value = ClickHouseBigDecimalValue.ofNull(); - break; - case Date: - case Date32: - value = ClickHouseDateValue.ofNull(); - break; - case DateTime: - case DateTime32: - case DateTime64: { - if (column == null) { - value = ClickHouseDateTimeValue.ofNull(0); - } else if (column.getTimeZone() == null) { - value = ClickHouseDateTimeValue.ofNull(column.getScale()); - } else { - value = ClickHouseOffsetDateTimeValue.ofNull(column.getScale(), column.getTimeZone()); - } - break; - } - case IPv4: - value = ClickHouseIpv4Value.ofNull(); - break; - case IPv6: - value = ClickHouseIpv6Value.ofNull(); - break; - case FixedString: - case String: - value = ClickHouseStringValue.ofNull(); - break; - case UUID: - value = ClickHouseUuidValue.ofNull(); - break; - case Point: - value = ClickHouseGeoPointValue.ofOrigin(); - break; - case Ring: - value = ClickHouseGeoRingValue.ofEmpty(); - break; - case Polygon: - value = ClickHouseGeoPolygonValue.ofEmpty(); - break; - case MultiPolygon: - value = ClickHouseGeoMultiPolygonValue.ofEmpty(); - break; - case AggregateFunction: - if (column != null) { - if (column.getAggregateFunction() == ClickHouseAggregateFunction.groupBitmap) { - value = ClickHouseBitmapValue.ofEmpty(column.getNestedColumns().get(0).getDataType()); + case Enum: + case Enum8: + case Int8: + value = ClickHouseByteValue.ofNull(); + break; + case UInt8: + case Enum16: + case Int16: + value = ClickHouseShortValue.ofNull(); + break; + case UInt16: + case Int32: + value = ClickHouseIntegerValue.ofNull(); + break; + case UInt32: + case IntervalYear: + case IntervalQuarter: + case IntervalMonth: + case IntervalWeek: + case IntervalDay: + case IntervalHour: + case IntervalMinute: + case IntervalSecond: + case Int64: + value = ClickHouseLongValue.ofNull(false); + break; + case UInt64: + value = ClickHouseLongValue.ofNull(true); + break; + case Int128: + case UInt128: + case Int256: + case UInt256: + value = ClickHouseBigIntegerValue.ofNull(); + break; + case Float32: + value = ClickHouseFloatValue.ofNull(); + break; + case Float64: + value = ClickHouseDoubleValue.ofNull(); + break; + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + value = ClickHouseBigDecimalValue.ofNull(); + break; + case Date: + case Date32: + value = ClickHouseDateValue.ofNull(); + break; + case DateTime: + case DateTime32: + case DateTime64: { + if (column == null) { + value = ClickHouseDateTimeValue.ofNull(0); + } else if (column.getTimeZone() == null) { + value = ClickHouseDateTimeValue.ofNull(column.getScale()); + } else { + value = ClickHouseOffsetDateTimeValue.ofNull(column.getScale(), column.getTimeZone()); } + break; } - break; - case Array: - if (column == null) { - value = ClickHouseArrayValue.ofEmpty(); - } else if (column.getArrayNestedLevel() > 1) { - value = ClickHouseArrayValue.of( - (Object[]) createPrimitiveArray(column.getArrayBaseColumn().getDataType().getPrimitiveClass(), - 0, column.getArrayNestedLevel())); - } else { - Class javaClass = column.getArrayBaseColumn().getDataType().getPrimitiveClass(); - if (byte.class == javaClass) { - value = ClickHouseByteArrayValue.ofEmpty(); - } else if (short.class == javaClass) { - value = ClickHouseShortArrayValue.ofEmpty(); - } else if (int.class == javaClass) { - value = ClickHouseIntArrayValue.ofEmpty(); - } else if (long.class == javaClass) { - value = ClickHouseLongArrayValue.ofEmpty(); - } else if (float.class == javaClass) { - value = ClickHouseFloatArrayValue.ofEmpty(); - } else if (double.class == javaClass) { - value = ClickHouseDoubleArrayValue.ofEmpty(); - } else { + case IPv4: + value = ClickHouseIpv4Value.ofNull(); + break; + case IPv6: + value = ClickHouseIpv6Value.ofNull(); + break; + case FixedString: + case String: + value = ClickHouseStringValue.ofNull(); + break; + case UUID: + value = ClickHouseUuidValue.ofNull(); + break; + case Point: + value = ClickHouseGeoPointValue.ofOrigin(); + break; + case Ring: + value = ClickHouseGeoRingValue.ofEmpty(); + break; + case Polygon: + value = ClickHouseGeoPolygonValue.ofEmpty(); + break; + case MultiPolygon: + value = ClickHouseGeoMultiPolygonValue.ofEmpty(); + break; + case AggregateFunction: + if (column != null) { + if (column.getAggregateFunction() == ClickHouseAggregateFunction.groupBitmap) { + value = ClickHouseBitmapValue.ofEmpty(column.getNestedColumns().get(0).getDataType()); + } + } + break; + case Array: + if (column == null) { value = ClickHouseArrayValue.ofEmpty(); + } else if (column.getArrayNestedLevel() > 1) { + value = ClickHouseArrayValue.of( + (Object[]) createPrimitiveArray( + column.getArrayBaseColumn().getDataType().getPrimitiveClass(), + 0, column.getArrayNestedLevel())); + } else { + Class javaClass = column.getArrayBaseColumn().getDataType().getPrimitiveClass(); + if (byte.class == javaClass) { + value = ClickHouseByteArrayValue.ofEmpty(); + } else if (short.class == javaClass) { + value = ClickHouseShortArrayValue.ofEmpty(); + } else if (int.class == javaClass) { + value = ClickHouseIntArrayValue.ofEmpty(); + } else if (long.class == javaClass) { + value = ClickHouseLongArrayValue.ofEmpty(); + } else if (float.class == javaClass) { + value = ClickHouseFloatArrayValue.ofEmpty(); + } else if (double.class == javaClass) { + value = ClickHouseDoubleArrayValue.ofEmpty(); + } else { + value = ClickHouseArrayValue.ofEmpty(); + } } - } - break; - case Map: - if (column == null) { - throw new IllegalArgumentException("column types for key and value are required"); - } - value = ClickHouseMapValue.ofEmpty(column.getKeyInfo().getDataType().getObjectClass(), - column.getValueInfo().getDataType().getObjectClass()); - break; - case Nested: - if (column == null) { - throw new IllegalArgumentException("nested column types are required"); - } - value = ClickHouseNestedValue.ofEmpty(column.getNestedColumns()); - break; - case Tuple: - value = ClickHouseTupleValue.of(); - break; - case Nothing: - value = ClickHouseEmptyValue.INSTANCE; - break; - default: - break; + break; + case Map: + if (column == null) { + throw new IllegalArgumentException("column types for key and value are required"); + } + value = ClickHouseMapValue.ofEmpty(column.getKeyInfo().getDataType().getObjectClass(), + column.getValueInfo().getDataType().getObjectClass()); + break; + case Nested: + if (column == null) { + throw new IllegalArgumentException("nested column types are required"); + } + value = ClickHouseNestedValue.ofEmpty(column.getNestedColumns()); + break; + case Tuple: + value = ClickHouseTupleValue.of(); + break; + case Nothing: + value = ClickHouseEmptyValue.INSTANCE; + break; + default: + break; } if (value == null) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java index 8858cef5b..49d9941ce 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseDataType; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValues; @@ -135,41 +136,6 @@ public static int getVarLongSize(long value) { return result; } - /** - * Read an unsigned byte from given input stream. - * - * @param input non-null input stream - * @return unnsigned byte which is always greater than or equal to zero - * @throws IOException when failed to read value from input stream or reached - * end of the stream - */ - public static int readUnsignedByte(InputStream input) throws IOException { - int value = input.read(); - if (value == -1) { - try { - input.close(); - } catch (IOException e) { - // ignore - } - - throw new EOFException(); - } - - return value; - } - - /** - * Read a byte from given input stream. - * - * @param input non-null input stream - * @return byte - * @throws IOException when failed to read value from input stream or reached - * end of the stream - */ - public static byte readByte(InputStream input) throws IOException { - return (byte) readUnsignedByte(input); - } - /** * Reads {@code length} bytes from given input stream. It behaves in the same * way as {@link java.io.DataInput#readFully(byte[])}. @@ -180,7 +146,7 @@ public static byte readByte(InputStream input) throws IOException { * @throws IOException when failed to read value from input stream, not able to * retrieve all bytes, or reached end of the stream */ - public static byte[] readBytes(InputStream input, int length) throws IOException { + public static byte[] readBytes(ClickHouseInputStream input, int length) throws IOException { int count = 0; byte[] bytes = new byte[length]; while (count < length) { @@ -225,7 +191,8 @@ public static void writeByteBuffer(OutputStream output, ByteBuffer buffer) throw * @throws IOException when failed to read value from input stream, not able to * retrieve all bytes, or reached end of the stream */ - public static ClickHouseBitmap readBitmap(InputStream input, ClickHouseDataType dataType) throws IOException { + public static ClickHouseBitmap readBitmap(ClickHouseInputStream input, ClickHouseDataType dataType) + throws IOException { return ClickHouseBitmap.deserialize(input, dataType); } @@ -286,7 +253,8 @@ public static char[] readCharacters(Reader input, int length) throws IOException } /** - * Read boolean from given input stream. It uses {@link #readByte(InputStream)} + * Read boolean from given input stream. It uses + * {@link ClickHouseInputStream#readByte()} * to get value and return {@code true} only when the value is {@code 1}. * * @param input non-null input stream @@ -294,8 +262,8 @@ public static char[] readCharacters(Reader input, int length) throws IOException * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static boolean readBoolean(InputStream input) throws IOException { - return ClickHouseChecker.between(readByte(input), ClickHouseValues.TYPE_BOOLEAN, 0, 1) == 1; + public static boolean readBoolean(ClickHouseInputStream input) throws IOException { + return ClickHouseChecker.between(input.readByte(), ClickHouseValues.TYPE_BOOLEAN, 0, 1) == 1; } /** @@ -333,20 +301,20 @@ public static void writeBoolean(OutputStream output, int value) throws IOExcepti * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static > T readEnum8(InputStream input, Class enumType) throws IOException { + public static > T readEnum8(ClickHouseInputStream input, Class enumType) throws IOException { return toEnum(readEnum8(input), enumType); } /** * Read enum value from given input stream. Same as - * {@link #readInt8(InputStream)}. + * {@link #readInt8(ClickHouseInputStream)}. * * @param input non-null input stream * @return enum value * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static byte readEnum8(InputStream input) throws IOException { + public static byte readEnum8(ClickHouseInputStream input) throws IOException { return readInt8(input); } @@ -386,20 +354,20 @@ public static > void writeEnum8(OutputStream output, T value) * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static > T readEnum16(InputStream input, Class enumType) throws IOException { + public static > T readEnum16(ClickHouseInputStream input, Class enumType) throws IOException { return toEnum(readEnum16(input), enumType); } /** * Read enum value from given input stream. Same as - * {@link #readInt16(InputStream)}. + * {@link #readInt16(ClickHouseInputStream)}. * * @param input non-null input stream * @return enum value * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static short readEnum16(InputStream input) throws IOException { + public static short readEnum16(ClickHouseInputStream input) throws IOException { return readInt16(input); } @@ -437,7 +405,7 @@ public static > void writeEnum16(OutputStream output, T value) * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static double[] readGeoPoint(InputStream input) throws IOException { + public static double[] readGeoPoint(ClickHouseInputStream input) throws IOException { return new double[] { readFloat64(input), readFloat64(input) }; } @@ -479,7 +447,7 @@ public static void writeGeoPoint(OutputStream output, double x, double y) throws * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static double[][] readGeoRing(InputStream input) throws IOException { + public static double[][] readGeoRing(ClickHouseInputStream input) throws IOException { int count = readVarInt(input); double[][] value = new double[count][2]; for (int i = 0; i < count; i++) { @@ -511,7 +479,7 @@ public static void writeGeoRing(OutputStream output, double[][] value) throws IO * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static double[][][] readGeoPolygon(InputStream input) throws IOException { + public static double[][][] readGeoPolygon(ClickHouseInputStream input) throws IOException { int count = readVarInt(input); double[][][] value = new double[count][][]; for (int i = 0; i < count; i++) { @@ -543,7 +511,7 @@ public static void writeGeoPolygon(OutputStream output, double[][][] value) thro * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static double[][][][] readGeoMultiPolygon(InputStream input) throws IOException { + public static double[][][][] readGeoMultiPolygon(ClickHouseInputStream input) throws IOException { int count = readVarInt(input); double[][][][] value = new double[count][][][]; for (int i = 0; i < count; i++) { @@ -569,14 +537,14 @@ public static void writeGeoMultiPolygon(OutputStream output, double[][][][] valu /** * Read null marker from input stream. Same as - * {@link #readBoolean(InputStream)}. + * {@link #readBoolean(ClickHouseInputStream)}. * * @param input non-null input stream * @return true if it's null; false otherwise * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static boolean readNull(InputStream input) throws IOException { + public static boolean readNull(ClickHouseInputStream input) throws IOException { return readBoolean(input); } @@ -610,7 +578,7 @@ public static void writeNonNull(OutputStream output) throws IOException { * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static Inet4Address readInet4Address(InputStream input) throws IOException { + public static Inet4Address readInet4Address(ClickHouseInputStream input) throws IOException { return (Inet4Address) InetAddress.getByAddress(reverse(readBytes(input, 4))); } @@ -634,7 +602,7 @@ public static void writeInet4Address(OutputStream output, Inet4Address value) th * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static Inet6Address readInet6Address(InputStream input) throws IOException { + public static Inet6Address readInet6Address(ClickHouseInputStream input) throws IOException { return Inet6Address.getByAddress(null, readBytes(input, 16), null); } @@ -651,15 +619,16 @@ public static void writeInet6Address(OutputStream output, Inet6Address value) th } /** - * Read a byte from given input stream. Same as {@link #readByte(InputStream)}. + * Read a byte from given input stream. Same as + * {@link ClickHouseInputStream#readByte()}. * * @param input non-null input stream * @return byte * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static byte readInt8(InputStream input) throws IOException { - return readByte(input); + public static byte readInt8(ClickHouseInputStream input) throws IOException { + return input.readByte(); } /** @@ -694,8 +663,8 @@ public static void writeInt8(OutputStream output, int value) throws IOException * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static short readUnsignedInt8(InputStream input) throws IOException { - return (short) (readByte(input) & 0xFFL); + public static short readUnsignedInt8(ClickHouseInputStream input) throws IOException { + return (short) (input.readByte() & 0xFF); } /** @@ -718,7 +687,7 @@ public static void writeUnsignedInt8(OutputStream output, int value) throws IOEx * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static short readInt16(InputStream input) throws IOException { + public static short readInt16(ClickHouseInputStream input) throws IOException { byte[] bytes = readBytes(input, 2); return (short) ((0xFF & bytes[0]) | (bytes[1] << 8)); } @@ -756,8 +725,8 @@ public static void writeInt16(OutputStream output, int value) throws IOException * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static int readUnsignedInt16(InputStream input) throws IOException { - return (int) (readInt16(input) & 0xFFFFL); + public static int readUnsignedInt16(ClickHouseInputStream input) throws IOException { + return (int) (readInt16(input) & 0xFFFF); } /** @@ -781,7 +750,7 @@ public static void writeUnsignedInt16(OutputStream output, int value) throws IOE * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static int readInt32(InputStream input) throws IOException { + public static int readInt32(ClickHouseInputStream input) throws IOException { byte[] bytes = readBytes(input, 4); return (0xFF & bytes[0]) | ((0xFF & bytes[1]) << 8) | ((0xFF & bytes[2]) << 16) | (bytes[3] << 24); @@ -808,8 +777,8 @@ public static void writeInt32(OutputStream output, int value) throws IOException * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static long readUnsignedInt32(InputStream input) throws IOException { - return readInt32(input) & 0xFFFFFFFFL; + public static long readUnsignedInt32(ClickHouseInputStream input) throws IOException { + return 0xFFFFFFFFL & readInt32(input); } /** @@ -833,7 +802,7 @@ public static void writeUnsignedInt32(OutputStream output, long value) throws IO * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static long readInt64(InputStream input) throws IOException { + public static long readInt64(ClickHouseInputStream input) throws IOException { byte[] bytes = readBytes(input, 8); return (0xFFL & bytes[0]) | ((0xFFL & bytes[1]) << 8) | ((0xFFL & bytes[2]) << 16) | ((0xFFL & bytes[3]) << 24) @@ -869,7 +838,7 @@ public static void writeInt64(OutputStream output, long value) throws IOExceptio * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigInteger readUnsignedInt64(InputStream input) throws IOException { + public static BigInteger readUnsignedInt64(ClickHouseInputStream input) throws IOException { return new BigInteger(1, reverse(readBytes(input, 8))); } @@ -909,7 +878,7 @@ public static void writeUnsignedInt64(OutputStream output, BigInteger value) thr * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigInteger readInt128(InputStream input) throws IOException { + public static BigInteger readInt128(ClickHouseInputStream input) throws IOException { return new BigInteger(reverse(readBytes(input, 16))); } @@ -933,7 +902,7 @@ public static void writeInt128(OutputStream output, BigInteger value) throws IOE * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigInteger readUnsignedInt128(InputStream input) throws IOException { + public static BigInteger readUnsignedInt128(ClickHouseInputStream input) throws IOException { return new BigInteger(1, reverse(readBytes(input, 16))); } @@ -958,7 +927,7 @@ public static void writeUnsignedInt128(OutputStream output, BigInteger value) th * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigInteger readInt256(InputStream input) throws IOException { + public static BigInteger readInt256(ClickHouseInputStream input) throws IOException { return new BigInteger(reverse(readBytes(input, 32))); } @@ -982,7 +951,7 @@ public static void writeInt256(OutputStream output, BigInteger value) throws IOE * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigInteger readUnsignedInt256(InputStream input) throws IOException { + public static BigInteger readUnsignedInt256(ClickHouseInputStream input) throws IOException { return new BigInteger(1, reverse(readBytes(input, 32))); } @@ -1007,7 +976,7 @@ public static void writeUnsignedInt256(OutputStream output, BigInteger value) th * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static float readFloat32(InputStream input) throws IOException { + public static float readFloat32(ClickHouseInputStream input) throws IOException { return Float.intBitsToFloat(readInt32(input)); } @@ -1031,7 +1000,7 @@ public static void writeFloat32(OutputStream output, float value) throws IOExcep * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static double readFloat64(InputStream input) throws IOException { + public static double readFloat64(ClickHouseInputStream input) throws IOException { return Double.longBitsToDouble(readInt64(input)); } @@ -1055,7 +1024,7 @@ public static void writeFloat64(OutputStream output, double value) throws IOExce * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static java.util.UUID readUuid(InputStream input) throws IOException { + public static java.util.UUID readUuid(ClickHouseInputStream input) throws IOException { return new java.util.UUID(readInt64(input), readInt64(input)); } @@ -1109,7 +1078,7 @@ public static void writeBigInteger(OutputStream output, BigInteger value, int le * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigDecimal readDecimal(InputStream input, int precision, int scale) throws IOException { + public static BigDecimal readDecimal(ClickHouseInputStream input, int precision, int scale) throws IOException { BigDecimal v; if (precision <= ClickHouseDataType.Decimal32.getMaxScale()) { @@ -1158,7 +1127,7 @@ public static void writeDecimal(OutputStream output, BigDecimal value, int preci * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigDecimal readDecimal32(InputStream input, int scale) throws IOException { + public static BigDecimal readDecimal32(ClickHouseInputStream input, int scale) throws IOException { return BigDecimal.valueOf(readInt32(input), ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, ClickHouseDataType.Decimal32.getMaxScale())); } @@ -1190,7 +1159,7 @@ public static void writeDecimal32(OutputStream output, BigDecimal value, int sca * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigDecimal readDecimal64(InputStream input, int scale) throws IOException { + public static BigDecimal readDecimal64(ClickHouseInputStream input, int scale) throws IOException { return BigDecimal.valueOf(readInt64(input), ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, ClickHouseDataType.Decimal64.getMaxScale())); } @@ -1223,7 +1192,7 @@ public static void writeDecimal64(OutputStream output, BigDecimal value, int sca * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigDecimal readDecimal128(InputStream input, int scale) throws IOException { + public static BigDecimal readDecimal128(ClickHouseInputStream input, int scale) throws IOException { return new BigDecimal(readInt128(input), ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, ClickHouseDataType.Decimal128.getMaxScale())); } @@ -1256,7 +1225,7 @@ public static void writeDecimal128(OutputStream output, BigDecimal value, int sc * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static BigDecimal readDecimal256(InputStream input, int scale) throws IOException { + public static BigDecimal readDecimal256(ClickHouseInputStream input, int scale) throws IOException { return new BigDecimal(readInt256(input), ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, ClickHouseDataType.Decimal256.getMaxScale())); } @@ -1289,7 +1258,7 @@ public static void writeDecimal256(OutputStream output, BigDecimal value, int sc * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDate readDate(InputStream input, TimeZone tz) throws IOException { + public static LocalDate readDate(ClickHouseInputStream input, TimeZone tz) throws IOException { return LocalDate.ofEpochDay(readUnsignedInt16(input)); } @@ -1316,7 +1285,7 @@ public static void writeDate(OutputStream output, LocalDate value, TimeZone tz) * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDate readDate32(InputStream input, TimeZone tz) throws IOException { + public static LocalDate readDate32(ClickHouseInputStream input, TimeZone tz) throws IOException { return LocalDate.ofEpochDay(readInt32(input)); } @@ -1343,7 +1312,7 @@ public static void writeDate32(OutputStream output, LocalDate value, TimeZone tz * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDateTime readDateTime(InputStream input, TimeZone tz) throws IOException { + public static LocalDateTime readDateTime(ClickHouseInputStream input, TimeZone tz) throws IOException { return readDateTime(input, 0, tz); } @@ -1357,7 +1326,7 @@ public static LocalDateTime readDateTime(InputStream input, TimeZone tz) throws * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDateTime readDateTime(InputStream input, int scale, TimeZone tz) throws IOException { + public static LocalDateTime readDateTime(ClickHouseInputStream input, int scale, TimeZone tz) throws IOException { return ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, ClickHouseDataType.DateTime64.getMaxScale()) == 0 ? readDateTime32(input, tz) : readDateTime64(input, scale, tz); @@ -1405,7 +1374,7 @@ public static void writeDateTime(OutputStream output, LocalDateTime value, int s * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDateTime readDateTime32(InputStream input, TimeZone tz) throws IOException { + public static LocalDateTime readDateTime32(ClickHouseInputStream input, TimeZone tz) throws IOException { long time = readUnsignedInt32(input); return LocalDateTime.ofInstant(Instant.ofEpochSecond(time < 0L ? 0L : time), @@ -1438,7 +1407,7 @@ public static void writeDateTime32(OutputStream output, LocalDateTime value, Tim * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDateTime readDateTime64(InputStream input, TimeZone tz) throws IOException { + public static LocalDateTime readDateTime64(ClickHouseInputStream input, TimeZone tz) throws IOException { return readDateTime64(input, 3, tz); } @@ -1452,7 +1421,7 @@ public static LocalDateTime readDateTime64(InputStream input, TimeZone tz) throw * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static LocalDateTime readDateTime64(InputStream input, int scale, TimeZone tz) throws IOException { + public static LocalDateTime readDateTime64(ClickHouseInputStream input, int scale, TimeZone tz) throws IOException { long value = readInt64(input); int nanoSeconds = 0; if (ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, 9) > 0) { @@ -1533,7 +1502,7 @@ public static void writeDateTime64(OutputStream output, LocalDateTime value, int * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static String readFixedString(InputStream input, int length) throws IOException { + public static String readFixedString(ClickHouseInputStream input, int length) throws IOException { return readFixedString(input, length, null); } @@ -1547,7 +1516,7 @@ public static String readFixedString(InputStream input, int length) throws IOExc * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static String readFixedString(InputStream input, int length, Charset charset) throws IOException { + public static String readFixedString(ClickHouseInputStream input, int length, Charset charset) throws IOException { byte[] bytes = readBytes(input, length); return new String(bytes, charset == null ? StandardCharsets.UTF_8 : charset); @@ -1595,7 +1564,7 @@ public static void writeFixedString(OutputStream output, String value, int lengt * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static String readString(InputStream input) throws IOException { + public static String readString(ClickHouseInputStream input) throws IOException { return readString(input, readVarInt(input), null); } @@ -1608,7 +1577,7 @@ public static String readString(InputStream input) throws IOException { * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static String readString(InputStream input, Charset charset) throws IOException { + public static String readString(ClickHouseInputStream input, Charset charset) throws IOException { return readString(input, readVarInt(input), charset); } @@ -1622,7 +1591,7 @@ public static String readString(InputStream input, Charset charset) throws IOExc * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static String readString(InputStream input, int length, Charset charset) throws IOException { + public static String readString(ClickHouseInputStream input, int length, Charset charset) throws IOException { return new String(readBytes(input, length), charset == null ? StandardCharsets.UTF_8 : charset); } @@ -1675,13 +1644,38 @@ public static void writeString(OutputStream output, String value, Charset charse * @throws IOException when failed to read value from input stream or reached * end of the stream */ + public static int readVarInt(ClickHouseInputStream input) throws IOException { + // https://github.com/ClickHouse/ClickHouse/blob/abe314feecd1647d7c2b952a25da7abf5c19f352/src/IO/VarInt.h#L126 + long result = 0L; + int shift = 0; + for (int i = 0; i < 9; i++) { + // gets 7 bits from next byte + int b = input.readUnsignedByte(); + result |= (b & 0x7F) << shift; + if ((b & 0x80) == 0) { + break; + } + shift += 7; + } + + return (int) result; + } + public static int readVarInt(InputStream input) throws IOException { // https://github.com/ClickHouse/ClickHouse/blob/abe314feecd1647d7c2b952a25da7abf5c19f352/src/IO/VarInt.h#L126 long result = 0L; int shift = 0; for (int i = 0; i < 9; i++) { // gets 7 bits from next byte - int b = readUnsignedByte(input); + int b = input.read(); + if (b == -1) { + try { + input.close(); + } catch (IOException e) { + // ignore error + } + throw new EOFException(); + } result |= (b & 0x7F) << shift; if ((b & 0x80) == 0) { break; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBigDecimalValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBigDecimalValue.java index d32b55ca3..23b2de938 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBigDecimalValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBigDecimalValue.java @@ -2,6 +2,7 @@ import java.math.BigDecimal; import java.math.BigInteger; +import java.math.RoundingMode; import java.net.Inet4Address; import java.net.Inet6Address; import java.time.LocalDate; @@ -114,9 +115,19 @@ public double asDouble() { return isNullOrEmpty() ? 0D : getValue().doubleValue(); } + @Override + public BigDecimal asBigDecimal() { + return getValue(); + } + @Override public BigDecimal asBigDecimal(int scale) { - return getValue().setScale(scale); + BigDecimal v = getValue(); + if (v != null && v.scale() != scale) { + v = v.setScale(scale, RoundingMode.DOWN); + } + + return v; } @Override diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java index a2481872a..fcb5790be 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java @@ -1,7 +1,7 @@ package com.clickhouse.client.data; +import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.nio.Buffer; import java.nio.ByteBuffer; @@ -9,6 +9,8 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; + +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseUtils; /** @@ -20,7 +22,7 @@ public class ClickHousePipedStream extends OutputStream { protected static final ByteBuffer EMPTY = ByteBuffer.wrap(new byte[0]); - static class Input extends InputStream { + static class Input extends ClickHouseInputStream { private final BlockingQueue queue; private final int timeout; @@ -80,6 +82,11 @@ public int available() throws IOException { return available; } + @Override + public boolean isClosed() { + return this.closed; + } + @Override public void close() throws IOException { // it's caller's responsiblity to consume all data in the queue, which will @@ -88,6 +95,24 @@ public void close() throws IOException { this.buffer = null; } + @Override + public byte readByte() throws IOException { + ensureOpen(); + + if (buffer == EMPTY || buffer.limit() == 0) { + close(); + throw new EOFException(); + } + + if (buffer.hasRemaining()) { + return buffer.get(); + } else { + updateBuffer(); + + return readByte(); + } + } + @Override public int read() throws IOException { ensureOpen(); @@ -210,7 +235,7 @@ private void updateBuffer() throws IOException { } } - public InputStream getInput() { + public ClickHouseInputStream getInput() { return new Input(queue, timeout); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java index 23bfdff71..e768d2ec4 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java @@ -2,7 +2,6 @@ import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.io.UncheckedIOException; import java.util.ArrayList; @@ -24,6 +23,7 @@ import com.clickhouse.client.ClickHouseDataType; import com.clickhouse.client.ClickHouseDeserializer; import com.clickhouse.client.ClickHouseFormat; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRecord; import com.clickhouse.client.ClickHouseSerializer; import com.clickhouse.client.ClickHouseUtils; @@ -103,7 +103,7 @@ private void writeArray(ClickHouseValue value, ClickHouseConfig config, ClickHou } private ClickHouseValue readArray(ClickHouseValue ref, ClickHouseConfig config, ClickHouseColumn nestedColumn, - ClickHouseColumn baseColumn, InputStream input, int length, int level) throws IOException { + ClickHouseColumn baseColumn, ClickHouseInputStream input, int length, int level) throws IOException { Class javaClass = baseColumn.getDataType().getPrimitiveClass(); if (level > 1 || !javaClass.isPrimitive()) { Object[] array = (Object[]) ClickHouseValues.createPrimitiveArray(javaClass, length, level); @@ -411,7 +411,7 @@ private MappedFunctions() { @SuppressWarnings("unchecked") public ClickHouseValue deserialize(ClickHouseValue ref, ClickHouseConfig config, ClickHouseColumn column, - InputStream input) throws IOException { + ClickHouseInputStream input) throws IOException { if (column.isNullable() && BinaryStreamUtils.readNull(input)) { return ref == null ? ClickHouseValues.newValue(column) : ref.resetToNullOrEmpty(); } @@ -540,7 +540,7 @@ protected List readColumns() throws IOException { return columns; } - public ClickHouseRowBinaryProcessor(ClickHouseConfig config, InputStream input, OutputStream output, + public ClickHouseRowBinaryProcessor(ClickHouseConfig config, ClickHouseInputStream input, OutputStream output, List columns, Map settings) throws IOException { super(config, input, output, columns, settings); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java index d33eabd88..38de95503 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java @@ -11,6 +11,7 @@ import com.clickhouse.client.ClickHouseDataProcessor; import com.clickhouse.client.ClickHouseDataStreamFactory; import com.clickhouse.client.ClickHouseFormat; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRecord; import com.clickhouse.client.ClickHouseResponse; import com.clickhouse.client.ClickHouseResponseSummary; @@ -28,27 +29,28 @@ public class ClickHouseStreamResponse implements ClickHouseResponse { protected static final List defaultTypes = Collections .singletonList(ClickHouseColumn.of("results", "Nullable(String)")); - public static ClickHouseResponse of(ClickHouseConfig config, InputStream input) throws IOException { + public static ClickHouseResponse of(ClickHouseConfig config, ClickHouseInputStream input) throws IOException { return of(config, input, null, null, null); } - public static ClickHouseResponse of(ClickHouseConfig config, InputStream input, Map settings) - throws IOException { + public static ClickHouseResponse of(ClickHouseConfig config, ClickHouseInputStream input, + Map settings) throws IOException { return of(config, input, settings, null, null); } - public static ClickHouseResponse of(ClickHouseConfig config, InputStream input, List columns) - throws IOException { + public static ClickHouseResponse of(ClickHouseConfig config, ClickHouseInputStream input, + List columns) throws IOException { return of(config, input, null, columns, null); } - public static ClickHouseResponse of(ClickHouseConfig config, InputStream input, Map settings, - List columns) throws IOException { + public static ClickHouseResponse of(ClickHouseConfig config, ClickHouseInputStream input, + Map settings, List columns) throws IOException { return of(config, input, settings, columns, null); } - public static ClickHouseResponse of(ClickHouseConfig config, InputStream input, Map settings, - List columns, ClickHouseResponseSummary summary) throws IOException { + public static ClickHouseResponse of(ClickHouseConfig config, ClickHouseInputStream input, + Map settings, List columns, ClickHouseResponseSummary summary) + throws IOException { return new ClickHouseStreamResponse(config, input, settings, columns, summary); } @@ -60,8 +62,9 @@ public static ClickHouseResponse of(ClickHouseConfig config, InputStream input, private boolean isClosed; - protected ClickHouseStreamResponse(ClickHouseConfig config, InputStream input, Map settings, - List columns, ClickHouseResponseSummary summary) throws IOException { + protected ClickHouseStreamResponse(ClickHouseConfig config, ClickHouseInputStream input, + Map settings, List columns, ClickHouseResponseSummary summary) + throws IOException { if (config == null || input == null) { throw new IllegalArgumentException("Non-null configuration and input stream are required"); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStringValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStringValue.java index c5e52d15e..7e65e817d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStringValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStringValue.java @@ -173,6 +173,11 @@ public Object asObject() { return value; } + @Override + public String asString() { + return value; + } + @Override public String asString(int length, Charset charset) { if (value != null && length > 0) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java index 56441ae68..dafe80ad4 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java @@ -1,7 +1,6 @@ package com.clickhouse.client.data; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; import java.util.Collections; @@ -14,6 +13,7 @@ import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseDataProcessor; import com.clickhouse.client.ClickHouseFormat; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRecord; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; @@ -111,15 +111,15 @@ public List readColumns() throws IOException { } switch (config.getFormat()) { - case TSVWithNames: - case TSVWithNamesAndTypes: - case TabSeparatedWithNames: - case TabSeparatedWithNamesAndTypes: - colDelimitter = (byte) 0x09; - break; - default: - colDelimitter = (byte) 0; - break; + case TSVWithNames: + case TSVWithNamesAndTypes: + case TabSeparatedWithNames: + case TabSeparatedWithNamesAndTypes: + colDelimitter = (byte) 0x09; + break; + default: + colDelimitter = (byte) 0; + break; } this.splitter = new StreamSplitter(input, rowDelimitter, config.getMaxBufferSize()); @@ -153,7 +153,7 @@ public List readColumns() throws IOException { return list; } - public ClickHouseTabSeparatedProcessor(ClickHouseConfig config, InputStream input, OutputStream output, + public ClickHouseTabSeparatedProcessor(ClickHouseConfig config, ClickHouseInputStream input, OutputStream output, List columns, Map settings) throws IOException { super(config, input, output, columns, settings); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java index dd9cfd924..446fc5265 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java @@ -3,7 +3,6 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.InputStream; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; @@ -19,6 +18,7 @@ import java.util.UUID; import com.clickhouse.client.ClickHouseDataType; +import com.clickhouse.client.ClickHouseInputStream; import org.roaringbitmap.RoaringBitmap; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; @@ -33,7 +33,8 @@ public class BinaryStreamUtilsTest { @DataProvider(name = "timeZoneProvider") private Object[][] getTimeZones() { return new Object[][] { new String[] { "Asia/Chongqing" }, new String[] { "America/Los_Angeles" }, - new String[] { "Europe/Moscow" }, new String[] { "Etc/UTC" }, new String[] { "Europe/Berlin" } }; + new String[] { "Europe/Moscow" }, new String[] { "Etc/UTC" }, + new String[] { "Europe/Berlin" } }; } protected static byte[] generateBytes(int... ints) { @@ -44,12 +45,12 @@ protected static byte[] generateBytes(int... ints) { return bytes; } - protected static InputStream generateInput(byte... values) { - return new ByteArrayInputStream(values); + protected static ClickHouseInputStream generateInput(byte... values) { + return ClickHouseInputStream.of(new ByteArrayInputStream(values)); } - protected static InputStream generateInput(int... values) { - return new ByteArrayInputStream(generateBytes(values)); + protected static ClickHouseInputStream generateInput(int... values) { + return ClickHouseInputStream.of(new ByteArrayInputStream(generateBytes(values))); } protected static byte[][] generateBytes(int byteLength, boolean unsigned) { @@ -156,7 +157,8 @@ public void testReadBoolean() throws IOException { Assert.assertEquals(BinaryStreamUtils.readBoolean(generateInput(1)), true); Assert.assertEquals(BinaryStreamUtils.readBoolean(generateInput(0)), false); - Assert.assertThrows(IllegalArgumentException.class, () -> BinaryStreamUtils.readBoolean(generateInput(2))); + Assert.assertThrows(IllegalArgumentException.class, + () -> BinaryStreamUtils.readBoolean(generateInput(2))); } @Test(groups = { "unit" }) @@ -168,7 +170,7 @@ public void testWriteBoolean() throws IOException { @Test(groups = { "unit" }) public void testReadByte() throws IOException { for (int b = Byte.MIN_VALUE; b <= Byte.MAX_VALUE; b++) { - Assert.assertEquals(BinaryStreamUtils.readByte(generateInput(b)), b); + Assert.assertEquals(generateInput(b).readByte(), b); Assert.assertEquals(BinaryStreamUtils.readInt8(generateInput(b)), b); } } @@ -193,10 +195,12 @@ public void testWriteEmptyBitmap() throws IOException { Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeBitmap(o, ClickHouseBitmap.empty())), generateBytes(0, 0)); - for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int8, ClickHouseDataType.UInt8, + for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int8, + ClickHouseDataType.UInt8, ClickHouseDataType.Int16, ClickHouseDataType.UInt16, ClickHouseDataType.Int32, ClickHouseDataType.UInt32, ClickHouseDataType.Int64, ClickHouseDataType.UInt64 }) { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeBitmap(o, ClickHouseBitmap.empty(t))), + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeBitmap(o, ClickHouseBitmap.empty(t))), generateBytes(0, 0)); } } @@ -204,7 +208,8 @@ public void testWriteEmptyBitmap() throws IOException { @Test(groups = { "unit" }) public void testWriteBitmap32() throws IOException { // cardinality <= 32 - for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int8, ClickHouseDataType.UInt8, + for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int8, + ClickHouseDataType.UInt8, ClickHouseDataType.Int16, ClickHouseDataType.UInt16, ClickHouseDataType.Int32, ClickHouseDataType.UInt32 }) { int[] values = newBitmapValues(t, 32); @@ -222,64 +227,102 @@ public void testWriteBitmap32() throws IOException { } } }); - for (Object bitmap : new Object[] { RoaringBitmap.bitmapOf(values), MutableRoaringBitmap.bitmapOf(values), + for (Object bitmap : new Object[] { RoaringBitmap.bitmapOf(values), + MutableRoaringBitmap.bitmapOf(values), ImmutableRoaringBitmap.bitmapOf(values) }) { byte[] actual = getWrittenBytes( - o -> BinaryStreamUtils.writeBitmap(o, ClickHouseBitmap.wrap(bitmap, t))); + o -> BinaryStreamUtils.writeBitmap(o, + ClickHouseBitmap.wrap(bitmap, t))); Assert.assertEquals(actual, expected); } } // cardinality > 32 int i = 0; - for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int8, ClickHouseDataType.UInt8, + for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int8, + ClickHouseDataType.UInt8, ClickHouseDataType.Int16, ClickHouseDataType.UInt16, ClickHouseDataType.Int32, ClickHouseDataType.UInt32 }) { int[] values = newBitmapValues(t, 33); byte[][] expected = new byte[][] { - generateBytes(0x01, 0x5A, 0x3A, 0x30, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x00, 0x00, 0x10, 0x00, - 0xFF, 0xFF, 0x0F, 0x00, 0x18, 0x00, 0x00, 0x00, 0x3A, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, - 0x00, 0x04, 0x00, 0x06, 0x00, 0x08, 0x00, 0x0A, 0x00, 0x0C, 0x00, 0x0E, 0x00, 0x10, 0x00, - 0x12, 0x00, 0x14, 0x00, 0x16, 0x00, 0x18, 0x00, 0x1A, 0x00, 0x1C, 0x00, 0x1E, 0x00, 0x20, - 0x00, 0xE1, 0xFF, 0xE3, 0xFF, 0xE5, 0xFF, 0xE7, 0xFF, 0xE9, 0xFF, 0xEB, 0xFF, 0xED, 0xFF, - 0xEF, 0xFF, 0xF1, 0xFF, 0xF3, 0xFF, 0xF5, 0xFF, 0xF7, 0xFF, 0xF9, 0xFF, 0xFB, 0xFF, 0xFD, + generateBytes(0x01, 0x5A, 0x3A, 0x30, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x10, 0x00, + 0xFF, 0xFF, 0x0F, 0x00, 0x18, 0x00, 0x00, 0x00, 0x3A, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x02, + 0x00, 0x04, 0x00, 0x06, 0x00, 0x08, 0x00, 0x0A, 0x00, 0x0C, + 0x00, 0x0E, 0x00, 0x10, 0x00, + 0x12, 0x00, 0x14, 0x00, 0x16, 0x00, 0x18, 0x00, 0x1A, 0x00, + 0x1C, 0x00, 0x1E, 0x00, 0x20, + 0x00, 0xE1, 0xFF, 0xE3, 0xFF, 0xE5, 0xFF, 0xE7, 0xFF, 0xE9, + 0xFF, 0xEB, 0xFF, 0xED, 0xFF, + 0xEF, 0xFF, 0xF1, 0xFF, 0xF3, 0xFF, 0xF5, 0xFF, 0xF7, 0xFF, + 0xF9, 0xFF, 0xFB, 0xFF, 0xFD, 0xFF, 0xFF, 0xFF), - generateBytes(0x01, 0x52, 0x3A, 0x30, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x20, 0x00, - 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x02, 0x00, 0x03, 0x00, 0x04, 0x00, 0x05, - 0x00, 0x06, 0x00, 0x07, 0x00, 0x08, 0x00, 0x09, 0x00, 0x0A, 0x00, 0x0B, 0x00, 0x0C, 0x00, - 0x0D, 0x00, 0x0E, 0x00, 0x0F, 0x00, 0x10, 0x00, 0x11, 0x00, 0x12, 0x00, 0x13, 0x00, 0x14, - 0x00, 0x15, 0x00, 0x16, 0x00, 0x17, 0x00, 0x18, 0x00, 0x19, 0x00, 0x1A, 0x00, 0x1B, 0x00, + generateBytes(0x01, 0x52, 0x3A, 0x30, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x20, 0x00, + 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x02, 0x00, + 0x03, 0x00, 0x04, 0x00, 0x05, + 0x00, 0x06, 0x00, 0x07, 0x00, 0x08, 0x00, 0x09, 0x00, 0x0A, + 0x00, 0x0B, 0x00, 0x0C, 0x00, + 0x0D, 0x00, 0x0E, 0x00, 0x0F, 0x00, 0x10, 0x00, 0x11, 0x00, + 0x12, 0x00, 0x13, 0x00, 0x14, + 0x00, 0x15, 0x00, 0x16, 0x00, 0x17, 0x00, 0x18, 0x00, 0x19, + 0x00, 0x1A, 0x00, 0x1B, 0x00, 0x1C, 0x00, 0x1D, 0x00, 0x1E, 0x00, 0x1F, 0x00, 0x20, 0x00), - generateBytes(0x01, 0x5A, 0x3A, 0x30, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x00, 0x00, 0x10, 0x00, - 0xFF, 0xFF, 0x0F, 0x00, 0x18, 0x00, 0x00, 0x00, 0x3A, 0x00, 0x00, 0x00, 0xFF, 0x00, 0x01, - 0x01, 0x03, 0x01, 0x05, 0x01, 0x07, 0x01, 0x09, 0x01, 0x0B, 0x01, 0x0D, 0x01, 0x0F, 0x01, - 0x11, 0x01, 0x13, 0x01, 0x15, 0x01, 0x17, 0x01, 0x19, 0x01, 0x1B, 0x01, 0x1D, 0x01, 0x1F, - 0x01, 0xE2, 0xFE, 0xE4, 0xFE, 0xE6, 0xFE, 0xE8, 0xFE, 0xEA, 0xFE, 0xEC, 0xFE, 0xEE, 0xFE, - 0xF0, 0xFE, 0xF2, 0xFE, 0xF4, 0xFE, 0xF6, 0xFE, 0xF8, 0xFE, 0xFA, 0xFE, 0xFC, 0xFE, 0xFE, + generateBytes(0x01, 0x5A, 0x3A, 0x30, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x10, 0x00, + 0xFF, 0xFF, 0x0F, 0x00, 0x18, 0x00, 0x00, 0x00, 0x3A, 0x00, + 0x00, 0x00, 0xFF, 0x00, 0x01, + 0x01, 0x03, 0x01, 0x05, 0x01, 0x07, 0x01, 0x09, 0x01, 0x0B, + 0x01, 0x0D, 0x01, 0x0F, 0x01, + 0x11, 0x01, 0x13, 0x01, 0x15, 0x01, 0x17, 0x01, 0x19, 0x01, + 0x1B, 0x01, 0x1D, 0x01, 0x1F, + 0x01, 0xE2, 0xFE, 0xE4, 0xFE, 0xE6, 0xFE, 0xE8, 0xFE, 0xEA, + 0xFE, 0xEC, 0xFE, 0xEE, 0xFE, + 0xF0, 0xFE, 0xF2, 0xFE, 0xF4, 0xFE, 0xF6, 0xFE, 0xF8, 0xFE, + 0xFA, 0xFE, 0xFC, 0xFE, 0xFE, 0xFE, 0x00, 0xFF), - generateBytes(0x01, 0x52, 0x3A, 0x30, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x20, 0x00, - 0x10, 0x00, 0x00, 0x00, 0xFF, 0x00, 0x00, 0x01, 0x01, 0x01, 0x02, 0x01, 0x03, 0x01, 0x04, - 0x01, 0x05, 0x01, 0x06, 0x01, 0x07, 0x01, 0x08, 0x01, 0x09, 0x01, 0x0A, 0x01, 0x0B, 0x01, - 0x0C, 0x01, 0x0D, 0x01, 0x0E, 0x01, 0x0F, 0x01, 0x10, 0x01, 0x11, 0x01, 0x12, 0x01, 0x13, - 0x01, 0x14, 0x01, 0x15, 0x01, 0x16, 0x01, 0x17, 0x01, 0x18, 0x01, 0x19, 0x01, 0x1A, 0x01, + generateBytes(0x01, 0x52, 0x3A, 0x30, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x20, 0x00, + 0x10, 0x00, 0x00, 0x00, 0xFF, 0x00, 0x00, 0x01, 0x01, 0x01, + 0x02, 0x01, 0x03, 0x01, 0x04, + 0x01, 0x05, 0x01, 0x06, 0x01, 0x07, 0x01, 0x08, 0x01, 0x09, + 0x01, 0x0A, 0x01, 0x0B, 0x01, + 0x0C, 0x01, 0x0D, 0x01, 0x0E, 0x01, 0x0F, 0x01, 0x10, 0x01, + 0x11, 0x01, 0x12, 0x01, 0x13, + 0x01, 0x14, 0x01, 0x15, 0x01, 0x16, 0x01, 0x17, 0x01, 0x18, + 0x01, 0x19, 0x01, 0x1A, 0x01, 0x1B, 0x01, 0x1C, 0x01, 0x1D, 0x01, 0x1E, 0x01, 0x1F, 0x01), - generateBytes(0x01, 0x5A, 0x3A, 0x30, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x00, 0x00, 0x10, 0x00, - 0xFF, 0xFF, 0x0F, 0x00, 0x18, 0x00, 0x00, 0x00, 0x3A, 0x00, 0x00, 0x00, 0xFD, 0x02, 0xFF, - 0x02, 0x01, 0x03, 0x03, 0x03, 0x05, 0x03, 0x07, 0x03, 0x09, 0x03, 0x0B, 0x03, 0x0D, 0x03, - 0x0F, 0x03, 0x11, 0x03, 0x13, 0x03, 0x15, 0x03, 0x17, 0x03, 0x19, 0x03, 0x1B, 0x03, 0x1D, - 0x03, 0xE4, 0xFC, 0xE6, 0xFC, 0xE8, 0xFC, 0xEA, 0xFC, 0xEC, 0xFC, 0xEE, 0xFC, 0xF0, 0xFC, - 0xF2, 0xFC, 0xF4, 0xFC, 0xF6, 0xFC, 0xF8, 0xFC, 0xFA, 0xFC, 0xFC, 0xFC, 0xFE, 0xFC, 0x00, + generateBytes(0x01, 0x5A, 0x3A, 0x30, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x10, 0x00, + 0xFF, 0xFF, 0x0F, 0x00, 0x18, 0x00, 0x00, 0x00, 0x3A, 0x00, + 0x00, 0x00, 0xFD, 0x02, 0xFF, + 0x02, 0x01, 0x03, 0x03, 0x03, 0x05, 0x03, 0x07, 0x03, 0x09, + 0x03, 0x0B, 0x03, 0x0D, 0x03, + 0x0F, 0x03, 0x11, 0x03, 0x13, 0x03, 0x15, 0x03, 0x17, 0x03, + 0x19, 0x03, 0x1B, 0x03, 0x1D, + 0x03, 0xE4, 0xFC, 0xE6, 0xFC, 0xE8, 0xFC, 0xEA, 0xFC, 0xEC, + 0xFC, 0xEE, 0xFC, 0xF0, 0xFC, + 0xF2, 0xFC, 0xF4, 0xFC, 0xF6, 0xFC, 0xF8, 0xFC, 0xFA, 0xFC, + 0xFC, 0xFC, 0xFE, 0xFC, 0x00, 0xFD, 0x02, 0xFD), - generateBytes(0x01, 0x52, 0x3A, 0x30, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x20, 0x00, - 0x10, 0x00, 0x00, 0x00, 0xFD, 0x02, 0xFE, 0x02, 0xFF, 0x02, 0x00, 0x03, 0x01, 0x03, 0x02, - 0x03, 0x03, 0x03, 0x04, 0x03, 0x05, 0x03, 0x06, 0x03, 0x07, 0x03, 0x08, 0x03, 0x09, 0x03, - 0x0A, 0x03, 0x0B, 0x03, 0x0C, 0x03, 0x0D, 0x03, 0x0E, 0x03, 0x0F, 0x03, 0x10, 0x03, 0x11, - 0x03, 0x12, 0x03, 0x13, 0x03, 0x14, 0x03, 0x15, 0x03, 0x16, 0x03, 0x17, 0x03, 0x18, 0x03, + generateBytes(0x01, 0x52, 0x3A, 0x30, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x20, 0x00, + 0x10, 0x00, 0x00, 0x00, 0xFD, 0x02, 0xFE, 0x02, 0xFF, 0x02, + 0x00, 0x03, 0x01, 0x03, 0x02, + 0x03, 0x03, 0x03, 0x04, 0x03, 0x05, 0x03, 0x06, 0x03, 0x07, + 0x03, 0x08, 0x03, 0x09, 0x03, + 0x0A, 0x03, 0x0B, 0x03, 0x0C, 0x03, 0x0D, 0x03, 0x0E, 0x03, + 0x0F, 0x03, 0x10, 0x03, 0x11, + 0x03, 0x12, 0x03, 0x13, 0x03, 0x14, 0x03, 0x15, 0x03, 0x16, + 0x03, 0x17, 0x03, 0x18, 0x03, 0x19, 0x03, 0x1A, 0x03, 0x1B, 0x03, 0x1C, 0x03, 0x1D, 0x03) }; - for (Object bitmap : new Object[] { RoaringBitmap.bitmapOf(values), MutableRoaringBitmap.bitmapOf(values), + for (Object bitmap : new Object[] { RoaringBitmap.bitmapOf(values), + MutableRoaringBitmap.bitmapOf(values), ImmutableRoaringBitmap.bitmapOf(values) }) { byte[] actual = getWrittenBytes( - o -> BinaryStreamUtils.writeBitmap(o, ClickHouseBitmap.wrap(bitmap, t))); + o -> BinaryStreamUtils.writeBitmap(o, + ClickHouseBitmap.wrap(bitmap, t))); Assert.assertEquals(actual, expected[i]); } i++; @@ -289,7 +332,8 @@ public void testWriteBitmap32() throws IOException { @Test(groups = { "unit" }) public void testWriteBitmap64() throws IOException { // cardinality <= 32 - for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int64, ClickHouseDataType.UInt64 }) { + for (ClickHouseDataType t : new ClickHouseDataType[] { ClickHouseDataType.Int64, + ClickHouseDataType.UInt64 }) { long[] values = newBitmap64Values(t, 32); byte[] expected = getWrittenBytes(o -> { BinaryStreamUtils.writeInt8(o, 0); @@ -301,7 +345,8 @@ public void testWriteBitmap64() throws IOException { for (Object bitmap : new Object[] { Roaring64Bitmap.bitmapOf(values), Roaring64NavigableMap.bitmapOf(values) }) { byte[] actual = getWrittenBytes( - o -> BinaryStreamUtils.writeBitmap(o, ClickHouseBitmap.wrap(bitmap, t))); + o -> BinaryStreamUtils.writeBitmap(o, + ClickHouseBitmap.wrap(bitmap, t))); Assert.assertEquals(actual, expected); } } @@ -361,7 +406,7 @@ public void testWriteBitmap64() throws IOException { @Test(groups = { "unit" }) public void testReadUnsignedByte() throws IOException { for (int i = 0; i < 0xFF; i++) { - Assert.assertEquals(BinaryStreamUtils.readUnsignedByte(generateInput(i)), i); + Assert.assertEquals(generateInput(i).readUnsignedByte(), i); Assert.assertEquals(BinaryStreamUtils.readUnsignedInt8(generateInput(i)), (short) i); } } @@ -370,7 +415,8 @@ public void testReadUnsignedByte() throws IOException { public void testWriteUnsignedByte() throws IOException { Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt8(o, 0)), generateBytes(0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt8(o, 1)), generateBytes(1)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt8(o, 255)), generateBytes(-1)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt8(o, 255)), + generateBytes(-1)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt8(o, 256))); Assert.assertThrows(IllegalArgumentException.class, @@ -401,8 +447,10 @@ public void testReadInt16() throws IOException { @Test(groups = { "unit" }) public void testWriteInt16() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, (short) 0)), generateBytes(0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, (short) 1)), generateBytes(1, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, (short) 0)), + generateBytes(0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, (short) 1)), + generateBytes(1, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, (short) -1)), generateBytes(0xFF, 0xFF)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, Short.MIN_VALUE)), @@ -412,7 +460,8 @@ public void testWriteInt16() throws IOException { Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, 0)), generateBytes(0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, 1)), generateBytes(1, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, -1)), generateBytes(0xFF, 0xFF)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, -1)), + generateBytes(0xFF, 0xFF)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeInt16(o, Short.MIN_VALUE - 1))); @@ -430,8 +479,10 @@ public void testReadUnsignedInt16() throws IOException { @Test(groups = { "unit" }) public void testWriteUnsignedInt16() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt16(o, 0)), generateBytes(0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt16(o, 1)), generateBytes(1, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt16(o, 0)), + generateBytes(0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt16(o, 1)), + generateBytes(1, 0)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt16(o, -1))); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt16(o, 0xFF)), @@ -449,14 +500,18 @@ public void testReadInt32() throws IOException { Assert.assertEquals(BinaryStreamUtils.readInt32(generateInput(0xFF, 0, 0, 0)), 0xFF); Assert.assertEquals(BinaryStreamUtils.readInt32(generateInput(0xFF, 0xFF, 0, 0)), 0xFFFF); Assert.assertEquals(BinaryStreamUtils.readInt32(generateInput(0xFF, 0xFF, 0xFF, 0)), 0xFFFFFF); - Assert.assertEquals(BinaryStreamUtils.readInt32(generateInput(0xFF, 0xFF, 0xFF, 0x7F)), Integer.MAX_VALUE); + Assert.assertEquals(BinaryStreamUtils.readInt32(generateInput(0xFF, 0xFF, 0xFF, 0x7F)), + Integer.MAX_VALUE); } @Test(groups = { "unit" }) public void testWriteInt32() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 0)), generateBytes(0, 0, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 1)), generateBytes(1, 0, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 0xFF)), generateBytes(0xFF, 0, 0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 0)), + generateBytes(0, 0, 0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 1)), + generateBytes(1, 0, 0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 0xFF)), + generateBytes(0xFF, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 0xFFFF)), generateBytes(0xFF, 0xFF, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt32(o, 0xFFFFFF)), @@ -475,7 +530,8 @@ public void testReadUnsignedInt32() throws IOException { Assert.assertEquals(BinaryStreamUtils.readUnsignedInt32(generateInput(0xFF, 0, 0, 0)), 0xFFL); Assert.assertEquals(BinaryStreamUtils.readUnsignedInt32(generateInput(0xFF, 0xFF, 0, 0)), 0xFFFFL); Assert.assertEquals(BinaryStreamUtils.readUnsignedInt32(generateInput(0xFF, 0xFF, 0xFF, 0)), 0xFFFFFFL); - Assert.assertEquals(BinaryStreamUtils.readUnsignedInt32(generateInput(0xFF, 0xFF, 0xFF, 0xFF)), 0xFFFFFFFFL); + Assert.assertEquals(BinaryStreamUtils.readUnsignedInt32(generateInput(0xFF, 0xFF, 0xFF, 0xFF)), + 0xFFFFFFFFL); } @Test(groups = { "unit" }) @@ -500,22 +556,30 @@ public void testWriteUnsignedInt32() throws IOException { @Test(groups = { "unit" }) public void testReadInt64() throws IOException { - Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0, 0, 0, 0, 0, 0, 0, 0x80)), Long.MIN_VALUE); + Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0, 0, 0, 0, 0, 0, 0, 0x80)), + Long.MIN_VALUE); Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0, 0, 0, 0, 0, 0, 0, 0)), 0L); Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0, 0, 0, 0, 0, 0, 0)), 0xFFL); Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0, 0, 0, 0, 0, 0)), 0xFFFFL); - Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0, 0, 0, 0, 0)), 0xFFFFFFL); + Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0, 0, 0, 0, 0)), + 0xFFFFFFL); Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0, 0, 0, 0)), 0xFFFFFFFFL); Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0, 0)), 0xFFFFFFFFFFL); - Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0)), + Assert.assertEquals( + BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0)), 0xFFFFFFFFFFFFL); - Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0)), + Assert.assertEquals( + BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0)), 0xFFFFFFFFFFFFFFL); - Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)), + Assert.assertEquals( + BinaryStreamUtils.readInt64( + generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)), -1L); - Assert.assertEquals(BinaryStreamUtils.readInt64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F)), + Assert.assertEquals( + BinaryStreamUtils.readInt64( + generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F)), Long.MAX_VALUE); } @@ -550,35 +614,43 @@ public void testReadUnsignedInt64() throws IOException { byte[] bytes = new byte[] { 0, 0, 0, 0, 0, 0, 0, 0 }; Assert.assertEquals(BinaryStreamUtils.readUnsignedInt64(generateInput(bytes)), BigInteger.ZERO); Assert.assertEquals( - BinaryStreamUtils.readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0, 0, 0, 0, 0, 0, 0))), + BinaryStreamUtils.readUnsignedInt64( + generateInput(bytes = generateBytes(0xFF, 0, 0, 0, 0, 0, 0, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( - BinaryStreamUtils.readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0, 0, 0, 0, 0, 0))), + BinaryStreamUtils.readUnsignedInt64( + generateInput(bytes = generateBytes(0xFF, 0xFF, 0, 0, 0, 0, 0, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( BinaryStreamUtils - .readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0, 0, 0, 0, 0))), + .readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, + 0, 0, 0, 0, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( BinaryStreamUtils - .readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0, 0, 0, 0))), + .readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, + 0xFF, 0, 0, 0, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( BinaryStreamUtils - .readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0, 0))), + .readUnsignedInt64(generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0, 0, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( BinaryStreamUtils.readUnsignedInt64( - generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0))), + generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( BinaryStreamUtils.readUnsignedInt64( - generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0))), + generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0))), new BigInteger(BinaryStreamUtils.reverse(bytes))); Assert.assertEquals( BinaryStreamUtils.readUnsignedInt64( - generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF))), + generateInput(bytes = generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF))), new BigInteger(1, BinaryStreamUtils.reverse(bytes))); } @@ -586,7 +658,8 @@ public void testReadUnsignedInt64() throws IOException { @Test(groups = { "unit" }) public void testWriteUnsignedInt64() throws IOException { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, BigInteger.valueOf(-1L)))); + () -> getWrittenBytes( + o -> BinaryStreamUtils.writeUnsignedInt64(o, BigInteger.valueOf(-1L)))); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, BigInteger.ZERO)), generateBytes(0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, BigInteger.ONE)), @@ -609,24 +682,33 @@ public void testWriteUnsignedInt64() throws IOException { generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, - new BigInteger(1, generateBytes(0, 0, 0, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)))), + new BigInteger(1, + generateBytes(0, 0, 0, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)))), generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, - new BigInteger(1, generateBytes(0, 0, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)))), + new BigInteger(1, + generateBytes(0, 0, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF)))), generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, - new BigInteger(1, generateBytes(0, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)))), + new BigInteger(1, + generateBytes(0, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF)))), generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, - new BigInteger(1, generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)))), + new BigInteger(1, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF)))), generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt64(o, - new BigInteger(1, generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)) - .add(BigInteger.ONE)))); + new BigInteger(1, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF)) + .add(BigInteger.ONE)))); } @Test(groups = { "unit" }) @@ -648,13 +730,15 @@ public void testWriteInt128() throws IOException { Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt128(o, BigInteger.ONE)), generateBytes(1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt128(o, BigInteger.valueOf(-1L))), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF)); byte[][] arr = generateBytes(Long.BYTES * 2, false); for (int i = 0; i < arr.length; i++) { byte[] bytes = arr[i]; - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt128(o, new BigInteger(bytes))), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeInt128(o, new BigInteger(bytes))), BinaryStreamUtils.reverse(bytes)); } } @@ -674,7 +758,8 @@ public void testReadUnsignedInt128() throws IOException { @Test(groups = { "unit" }) public void testWriteUnsignedInt128() throws IOException { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt128(o, BigInteger.valueOf(-1L)))); + () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt128(o, + BigInteger.valueOf(-1L)))); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt128(o, BigInteger.ZERO)), generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt128(o, BigInteger.ONE)), @@ -684,7 +769,8 @@ public void testWriteUnsignedInt128() throws IOException { for (int i = 0; i < arr.length; i++) { byte[] bytes = arr[i]; Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt128(o, new BigInteger(1, bytes))), + getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt128(o, + new BigInteger(1, bytes))), BinaryStreamUtils.reverse(bytes)); } } @@ -703,19 +789,26 @@ public void testReadInt256() throws IOException { @Test(groups = { "unit" }) public void testWriteInt256() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, BigInteger.ZERO)), generateBytes(0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, BigInteger.ONE)), generateBytes(1, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, BigInteger.ZERO)), + generateBytes(0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, BigInteger.ONE)), + generateBytes(1, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, BigInteger.valueOf(-1L))), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)); byte[][] arr = generateBytes(Long.BYTES * 4, false); for (int i = 0; i < arr.length; i++) { byte[] bytes = arr[i]; - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, new BigInteger(bytes))), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeInt256(o, new BigInteger(bytes))), BinaryStreamUtils.reverse(bytes)); } } @@ -735,19 +828,23 @@ public void testReadUnsignedInt256() throws IOException { @Test(groups = { "unit" }) public void testWriteUnsignedInt256() throws IOException { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt256(o, BigInteger.valueOf(-1L)))); + () -> getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt256(o, + BigInteger.valueOf(-1L)))); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt256(o, BigInteger.ZERO)), - generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt256(o, BigInteger.ONE)), - generateBytes(1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + generateBytes(1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0)); byte[][] arr = generateBytes(Long.BYTES * 4, true); for (int i = 0; i < arr.length; i++) { byte[] bytes = arr[i]; Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt256(o, new BigInteger(1, bytes))), + getWrittenBytes(o -> BinaryStreamUtils.writeUnsignedInt256(o, + new BigInteger(1, bytes))), BinaryStreamUtils.reverse(bytes)); } } @@ -755,32 +852,40 @@ public void testWriteUnsignedInt256() throws IOException { @Test(groups = { "unit" }) public void testReadDate() throws IOException { Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(-1, 0), null), LocalDate.ofEpochDay(255)); - Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0, 0x80), null), LocalDate.ofEpochDay(0x8000)); + Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0, 0x80), null), + LocalDate.ofEpochDay(0x8000)); Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0, 0), null), LocalDate.ofEpochDay(0)); Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(1, 0), null), LocalDate.ofEpochDay(1)); Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0xFF, 0x7F), null), LocalDate.ofEpochDay(Short.MAX_VALUE)); - Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0xFF, 0xFF), null), LocalDate.ofEpochDay(0xFFFF)); + Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0xFF, 0xFF), null), + LocalDate.ofEpochDay(0xFFFF)); - Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0x9E, 0x49), null), LocalDate.of(2021, 8, 7)); + Assert.assertEquals(BinaryStreamUtils.readDate(generateInput(0x9E, 0x49), null), + LocalDate.of(2021, 8, 7)); } @Test(groups = { "unit" }) public void testWriteDate() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(255), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(255), null)), generateBytes(-1, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(0x8000), null)), + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(0x8000), null)), generateBytes(0, 0x80)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(0), null)), generateBytes(0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(1), null)), generateBytes(1, 0)); Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(Short.MAX_VALUE), null)), + getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, + LocalDate.ofEpochDay(Short.MAX_VALUE), null)), generateBytes(0xFF, 0x7F)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(0xFFFF), null)), + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDate(o, LocalDate.ofEpochDay(0xFFFF), null)), generateBytes(0xFF, 0xFF)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.of(2021, 8, 7), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate(o, LocalDate.of(2021, 8, 7), null)), generateBytes(0x9E, 0x49)); } @@ -788,8 +893,10 @@ public void testWriteDate() throws IOException { public void testReadDate32() throws IOException { Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0xFF, 0xFF, 0xFF, 0xFF), null), LocalDate.ofEpochDay(-1)); - Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0, 0, 0, 0), null), LocalDate.ofEpochDay(0)); - Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(1, 0, 0, 0), null), LocalDate.ofEpochDay(1)); + Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0, 0, 0, 0), null), + LocalDate.ofEpochDay(0)); + Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(1, 0, 0, 0), null), + LocalDate.ofEpochDay(1)); Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0x17, 0x61, 0, 0), null), LocalDate.of(2038, 1, 19)); @@ -804,8 +911,10 @@ public void testReadDate32WithTimeZone(String timeZoneId) throws IOException { TimeZone tz = TimeZone.getTimeZone(timeZoneId); Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0xFF, 0xFF, 0xFF, 0xFF), tz), LocalDate.ofEpochDay(-1)); - Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0, 0, 0, 0), tz), LocalDate.ofEpochDay(0)); - Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(1, 0, 0, 0), tz), LocalDate.ofEpochDay(1)); + Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0, 0, 0, 0), tz), + LocalDate.ofEpochDay(0)); + Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(1, 0, 0, 0), tz), + LocalDate.ofEpochDay(1)); Assert.assertEquals(BinaryStreamUtils.readDate32(generateInput(0x17, 0x61, 0, 0), tz), LocalDate.of(2038, 1, 19)); @@ -817,47 +926,61 @@ public void testReadDate32WithTimeZone(String timeZoneId) throws IOException { @Test(groups = { "unit" }) public void testWriteDate32() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(-1), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(-1), null)), generateBytes(0xFF, 0xFF, 0xFF, 0xFF)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(0), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(0), null)), generateBytes(0, 0, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(1), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(1), null)), generateBytes(1, 0, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2038, 1, 19), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2038, 1, 19), null)), generateBytes(0x17, 0x61, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(1925, 1, 1), null)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(1925, 1, 1), null)), generateBytes(0xCC, 0xBF, 0xFF, 0xFF)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2283, 11, 11), null)), + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2283, 11, 11), null)), generateBytes(0xCB, 0xBF, 1, 0)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( - o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), null))); + o -> BinaryStreamUtils.writeDate32(o, + LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), null))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( - o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), null))); + o -> BinaryStreamUtils.writeDate32(o, + LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), null))); } @Test(dataProvider = "timeZoneProvider", groups = { "unit" }) public void testWriteDate32WithTimeZone(String timeZoneId) throws IOException { TimeZone tz = TimeZone.getTimeZone(timeZoneId); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(-1), tz)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(-1), tz)), generateBytes(0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(0), tz)), generateBytes(0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.ofEpochDay(1), tz)), generateBytes(1, 0, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2038, 1, 19), tz)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2038, 1, 19), tz)), generateBytes(0x17, 0x61, 0, 0)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(1925, 1, 1), tz)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(1925, 1, 1), tz)), generateBytes(0xCC, 0xBF, 0xFF, 0xFF)); - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2283, 11, 11), tz)), + Assert.assertEquals( + getWrittenBytes(o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2283, 11, 11), tz)), generateBytes(0xCB, 0xBF, 1, 0)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( - o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), tz))); + o -> BinaryStreamUtils.writeDate32(o, + LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), tz))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( - o -> BinaryStreamUtils.writeDate32(o, LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), tz))); + o -> BinaryStreamUtils.writeDate32(o, + LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), tz))); } @Test(groups = { "unit" }) @@ -865,7 +988,8 @@ public void testReadDateTime32() throws IOException { Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(-1, 0, 0, 0), null), LocalDateTime.ofEpochSecond(255, 0, ZoneOffset.UTC)); Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(0, 0, 0, 0x80), null), LocalDateTime - .ofEpochSecond(new BigInteger(1, generateBytes(0x80, 0, 0, 0)).longValue(), 0, ZoneOffset.UTC)); + .ofEpochSecond(new BigInteger(1, generateBytes(0x80, 0, 0, 0)).longValue(), 0, + ZoneOffset.UTC)); Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(0, 0, 0, 0), null), LocalDateTime.ofEpochSecond(0L, 0, ZoneOffset.UTC)); Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(1, 0, 0, 0), null), @@ -882,8 +1006,11 @@ public void testReadDateTime32WithTimeZone(String timeZoneId) throws IOException TimeZone tz = TimeZone.getTimeZone(timeZoneId); Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(-1, 0, 0, 0), tz), LocalDateTime.ofInstant(Instant.ofEpochSecond(255L), tz.toZoneId())); - Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(0, 0, 0, 0x80), tz), LocalDateTime.ofInstant( - Instant.ofEpochSecond(new BigInteger(1, generateBytes(0x80, 0, 0, 0)).longValue()), tz.toZoneId())); + Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(0, 0, 0, 0x80), tz), + LocalDateTime.ofInstant( + Instant.ofEpochSecond(new BigInteger(1, generateBytes(0x80, 0, 0, 0)) + .longValue()), + tz.toZoneId())); Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(0, 0, 0, 0), tz), LocalDateTime.ofInstant(Instant.ofEpochSecond(0L), tz.toZoneId())); Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(1, 0, 0, 0), tz), @@ -893,40 +1020,51 @@ public void testReadDateTime32WithTimeZone(String timeZoneId) throws IOException Assert.assertEquals(BinaryStreamUtils.readDateTime32(generateInput(0x2D, 0x9A, 0x0E, 0x61), tz), LocalDateTime.ofInstant( - Instant.ofEpochSecond(new BigInteger(1, generateBytes(0x61, 0x0E, 0x9A, 0x2D)).longValue()), + Instant.ofEpochSecond( + new BigInteger(1, generateBytes(0x61, 0x0E, 0x9A, 0x2D)) + .longValue()), tz.toZoneId())); } @Test(groups = { "unit" }) public void testWriteDateTime32() throws IOException { Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofEpochSecond(255, 0, ZoneOffset.UTC), null)), + o -> BinaryStreamUtils.writeDateTime32(o, + LocalDateTime.ofEpochSecond(255, 0, ZoneOffset.UTC), null)), generateBytes(-1, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofEpochSecond( - new BigInteger(1, generateBytes(0x80, 0, 0, 0)).longValue(), 0, ZoneOffset.UTC), null)), + new BigInteger(1, generateBytes(0x80, 0, 0, 0)).longValue(), 0, + ZoneOffset.UTC), null)), generateBytes(0, 0, 0, 0x80)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofEpochSecond(0L, 0, ZoneOffset.UTC), null)), + o -> BinaryStreamUtils.writeDateTime32(o, + LocalDateTime.ofEpochSecond(0L, 0, ZoneOffset.UTC), null)), generateBytes(0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofEpochSecond(1L, 0, ZoneOffset.UTC), null)), + o -> BinaryStreamUtils.writeDateTime32(o, + LocalDateTime.ofEpochSecond(1L, 0, ZoneOffset.UTC), null)), generateBytes(1, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, - LocalDateTime.ofEpochSecond(Integer.MAX_VALUE, 0, ZoneOffset.UTC), null)), + LocalDateTime.ofEpochSecond(Integer.MAX_VALUE, 0, ZoneOffset.UTC), + null)), generateBytes(0xFF, 0xFF, 0xFF, 0x7F)); Assert.assertEquals( getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.of(2021, 8, 7, 14, 35, 25), null)), + o -> BinaryStreamUtils.writeDateTime32(o, + LocalDateTime.of(2021, 8, 7, 14, 35, 25), null)), generateBytes(0x2D, 0x9A, 0x0E, 0x61)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofEpochSecond(-1L, 0, ZoneOffset.UTC), null))); + o -> BinaryStreamUtils.writeDateTime32(o, + LocalDateTime.ofEpochSecond(-1L, 0, ZoneOffset.UTC), null))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, - LocalDateTime.ofEpochSecond(BinaryStreamUtils.DATETIME_MAX + 1, 0, ZoneOffset.UTC), null))); + LocalDateTime.ofEpochSecond(BinaryStreamUtils.DATETIME_MAX + 1, 0, + ZoneOffset.UTC), + null))); } @Test(dataProvider = "timeZoneProvider", groups = { "unit" }) @@ -934,11 +1072,13 @@ public void testWriteDateTime32WithTimeZone(String timeZoneId) throws IOExceptio TimeZone tz = TimeZone.getTimeZone(timeZoneId); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(255L), tz.toZoneId()), tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(255L), tz.toZoneId()), + tz)), generateBytes(-1, 0, 0, 0)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofInstant( - Instant.ofEpochSecond(new BigInteger(1, generateBytes(0x80, 0, 0, 0)).longValue()), + Instant.ofEpochSecond(new BigInteger(1, generateBytes(0x80, 0, 0, 0)) + .longValue()), tz.toZoneId()), tz)), generateBytes(0, 0, 0, 0x80)); Assert.assertEquals( @@ -951,20 +1091,27 @@ public void testWriteDateTime32WithTimeZone(String timeZoneId) throws IOExceptio generateBytes(1, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(Integer.MAX_VALUE), tz.toZoneId()), tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(Integer.MAX_VALUE), + tz.toZoneId()), + tz)), generateBytes(0xFF, 0xFF, 0xFF, 0x7F)); Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime.ofInstant( - Instant.ofEpochSecond(new BigInteger(1, generateBytes(0x61, 0x0E, 0x9A, 0x2D)).longValue()), + Instant.ofEpochSecond( + new BigInteger(1, generateBytes(0x61, 0x0E, 0x9A, 0x2D)) + .longValue()), tz.toZoneId()), tz)), generateBytes(0x2D, 0x9A, 0x0E, 0x61)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils - .writeDateTime32(o, LocalDateTime.ofInstant(Instant.ofEpochSecond(-1L), tz.toZoneId()), tz))); + .writeDateTime32(o, LocalDateTime.ofInstant(Instant.ofEpochSecond(-1L), tz.toZoneId()), + tz))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeDateTime32(o, LocalDateTime - .ofInstant(Instant.ofEpochSecond(BinaryStreamUtils.DATETIME_MAX + 1), tz.toZoneId()), tz))); + .ofInstant(Instant.ofEpochSecond(BinaryStreamUtils.DATETIME_MAX + 1), + tz.toZoneId()), + tz))); } @Test(groups = { "unit" }) @@ -1000,18 +1147,21 @@ public void testReadDateTime64WithTimeZone(String timeZoneId) throws IOException Assert.assertEquals(BinaryStreamUtils.readDateTime64(generateInput(-1, 0, 0, 0, 0, 0, 0, 0), 0, tz), LocalDateTime.ofInstant(Instant.ofEpochSecond(255L), tz.toZoneId())); Assert.assertEquals( - BinaryStreamUtils.readDateTime64(generateInput(0xF6, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 1, tz), + BinaryStreamUtils.readDateTime64( + generateInput(0xF6, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 1, tz), LocalDateTime.ofEpochSecond(-1L, 0, ZoneOffset.UTC).atOffset(ZoneOffset.UTC) .atZoneSameInstant(tz.toZoneId()).toLocalDateTime()); Assert.assertEquals( - BinaryStreamUtils.readDateTime64(generateInput(0xF5, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 1, tz), + BinaryStreamUtils.readDateTime64( + generateInput(0xF5, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 1, tz), LocalDateTime.ofEpochSecond(-2L, 900000000, ZoneOffset.UTC).atOffset(ZoneOffset.UTC) .atZoneSameInstant(tz.toZoneId()).toLocalDateTime()); // Actually query "select toDateTime64(-1.000000001::Decimal64(9), 9)" returns: // 1969-12-31 23:59:59.000000001 // see https://github.com/ClickHouse/ClickHouse/issues/29386 Assert.assertEquals( - BinaryStreamUtils.readDateTime64(generateInput(0xFF, 0x35, 0x65, 0xC4, 0xFF, 0xFF, 0xFF, 0xFF), 9, tz), + BinaryStreamUtils.readDateTime64( + generateInput(0xFF, 0x35, 0x65, 0xC4, 0xFF, 0xFF, 0xFF, 0xFF), 9, tz), LocalDateTime.ofEpochSecond(-2L, 999999999, ZoneOffset.UTC).atOffset(ZoneOffset.UTC) .atZoneSameInstant(tz.toZoneId()).toLocalDateTime()); Assert.assertEquals(BinaryStreamUtils.readDateTime64(generateInput(0, 0, 0, 0, 0, 0, 0, 0), 0, tz), @@ -1048,16 +1198,20 @@ public void testWriteDateTime64() throws IOException { LocalDateTime.ofEpochSecond(-2L, 999999999, ZoneOffset.UTC), 9, null)), generateBytes(0xFF, 0x35, 0x65, 0xC4, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime64(o, LocalDateTime.ofEpochSecond(0L, 0, ZoneOffset.UTC), 0, null)), + o -> BinaryStreamUtils.writeDateTime64(o, + LocalDateTime.ofEpochSecond(0L, 0, ZoneOffset.UTC), 0, null)), generateBytes(0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime64(o, LocalDateTime.ofEpochSecond(1L, 0, ZoneOffset.UTC), 0, null)), + o -> BinaryStreamUtils.writeDateTime64(o, + LocalDateTime.ofEpochSecond(1L, 0, ZoneOffset.UTC), 0, null)), generateBytes(1, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime64(o, LocalDateTime.ofEpochSecond(1L, 0, ZoneOffset.UTC), 1, null)), + o -> BinaryStreamUtils.writeDateTime64(o, + LocalDateTime.ofEpochSecond(1L, 0, ZoneOffset.UTC), 1, null)), generateBytes(0x0A, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeDateTime64(o, LocalDateTime.ofEpochSecond(0L, 1, ZoneOffset.UTC), 9, null)), + o -> BinaryStreamUtils.writeDateTime64(o, + LocalDateTime.ofEpochSecond(0L, 1, ZoneOffset.UTC), 9, null)), generateBytes(1, 0, 0, 0, 0, 0, 0, 0)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils @@ -1067,10 +1221,14 @@ public void testWriteDateTime64() throws IOException { Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.of(LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), LocalTime.MAX), null))); + LocalDateTime.of(LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), + LocalTime.MAX), + null))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.of(LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), LocalTime.MIN), null))); + LocalDateTime.of(LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), + LocalTime.MIN), + null))); } @Test(dataProvider = "timeZoneProvider", groups = { "unit" }) @@ -1078,53 +1236,77 @@ public void testWriteDateTime64WithTimeZone(String timeZoneId) throws IOExceptio TimeZone tz = TimeZone.getTimeZone(timeZoneId); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(255L), tz.toZoneId()), 0, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(255L), tz.toZoneId()), 0, + tz)), generateBytes(-1, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(-1L), tz.toZoneId()), 1, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(-1L), tz.toZoneId()), 1, + tz)), generateBytes(0xF6, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(-2L, 900000000), tz.toZoneId()), 1, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(-2L, 900000000), + tz.toZoneId()), + 1, tz)), generateBytes(0xF5, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(-2L, 999999999), tz.toZoneId()), 9, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(-2L, 999999999), + tz.toZoneId()), + 9, tz)), generateBytes(0xFF, 0x35, 0x65, 0xC4, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 0), tz.toZoneId()), 0, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 0), tz.toZoneId()), 0, + tz)), generateBytes(0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(1L, 0), tz.toZoneId()), 0, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(1L, 0), tz.toZoneId()), 0, + tz)), generateBytes(1, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(1L, 0), tz.toZoneId()), 1, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(1L, 0), tz.toZoneId()), 1, + tz)), generateBytes(0x0A, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 1), tz.toZoneId()), 9, tz)), + LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 1), tz.toZoneId()), 9, + tz)), generateBytes(1, 0, 0, 0, 0, 0, 0, 0)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils - .writeDateTime64(o, LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 0), tz.toZoneId()), -1, tz))); + .writeDateTime64(o, + LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 0), tz.toZoneId()), + -1, tz))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils - .writeDateTime64(o, LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 0), tz.toZoneId()), 10, tz))); + .writeDateTime64(o, + LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, 0), tz.toZoneId()), + 10, tz))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.of(LocalDate.of(1925, 1, 1).minus(1L, ChronoUnit.DAYS), LocalTime.MAX) - .atOffset(ZoneOffset.UTC).atZoneSameInstant(tz.toZoneId()).toLocalDateTime(), + LocalDateTime.of( + LocalDate.of(1925, 1, 1).minus(1L, + ChronoUnit.DAYS), + LocalTime.MAX) + .atOffset(ZoneOffset.UTC) + .atZoneSameInstant(tz.toZoneId()) + .toLocalDateTime(), tz))); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes( o -> BinaryStreamUtils.writeDateTime64(o, - LocalDateTime.of(LocalDate.of(2283, 11, 11).plus(1L, ChronoUnit.DAYS), LocalTime.MIN) - .atOffset(ZoneOffset.UTC).atZoneSameInstant(tz.toZoneId()).toLocalDateTime(), + LocalDateTime.of( + LocalDate.of(2283, 11, 11).plus(1L, + ChronoUnit.DAYS), + LocalTime.MIN) + .atOffset(ZoneOffset.UTC) + .atZoneSameInstant(tz.toZoneId()) + .toLocalDateTime(), tz))); } @@ -1161,9 +1343,11 @@ public void testWriteDecimal32() throws IOException { BigDecimal d = new BigDecimal(new BigInteger(BinaryStreamUtils.reverse(bytes)), scale); if (d.toString().replaceAll("[-.]", "").replaceAll("^0+", "").length() > 9) { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, d, scale))); + () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, + d, scale))); } else { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, d, scale)), b); + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDecimal32(o, d, scale)), b); } } } @@ -1171,10 +1355,12 @@ public void testWriteDecimal32() throws IOException { Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, new BigDecimal("-1"), 0)), generateBytes(0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, new BigDecimal("0.9999999999"), 9)), + getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, new BigDecimal("0.9999999999"), + 9)), generateBytes(0xFF, 0xC9, 0x9A, 0x3B)); Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, new BigDecimal("-0.9999999999"), 9)), + getWrittenBytes(o -> BinaryStreamUtils.writeDecimal32(o, + new BigDecimal("-0.9999999999"), 9)), generateBytes(0x01, 0x36, 0x65, 0xC4)); Assert.assertThrows(IllegalArgumentException.class, @@ -1198,13 +1384,16 @@ public void testReadDecimal64() throws IOException { } Assert.assertEquals( - BinaryStreamUtils.readDecimal64(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 0), + BinaryStreamUtils.readDecimal64( + generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 0), new BigDecimal("-1")); Assert.assertEquals( - BinaryStreamUtils.readDecimal64(generateInput(0, 0, 0x64, 0xA7, 0xB3, 0xB6, 0xE0, 0x0D), 18), + BinaryStreamUtils.readDecimal64(generateInput(0, 0, 0x64, 0xA7, 0xB3, 0xB6, 0xE0, 0x0D), + 18), new BigDecimal("1.000000000000000000")); Assert.assertEquals( - BinaryStreamUtils.readDecimal64(generateInput(0, 0, 0x9C, 0x58, 0x4C, 0x49, 0x1F, 0xF2), 18), + BinaryStreamUtils.readDecimal64(generateInput(0, 0, 0x9C, 0x58, 0x4C, 0x49, 0x1F, 0xF2), + 18), new BigDecimal("-1.000000000000000000")); } @@ -1220,9 +1409,11 @@ public void testWriteDecimal64() throws IOException { BigDecimal d = new BigDecimal(new BigInteger(BinaryStreamUtils.reverse(bytes)), scale); if (d.toString().replaceAll("[-.]", "").replaceAll("^0+", "").length() > 18) { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal64(o, d, scale))); + () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal64(o, + d, scale))); } else { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal64(o, d, scale)), b); + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDecimal64(o, d, scale)), b); } } } @@ -1230,11 +1421,13 @@ public void testWriteDecimal64() throws IOException { Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal64(o, new BigDecimal("-1"), 0)), generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeDecimal64(o, new BigDecimal("0.99999999999999999999"), 18)), + getWrittenBytes(o -> BinaryStreamUtils.writeDecimal64(o, + new BigDecimal("0.99999999999999999999"), 18)), generateBytes(0xFF, 0xFF, 0x63, 0xA7, 0xB3, 0xB6, 0xE0, 0x0D)); Assert.assertEquals( getWrittenBytes( - o -> BinaryStreamUtils.writeDecimal64(o, new BigDecimal("-0.99999999999999999999"), 18)), + o -> BinaryStreamUtils.writeDecimal64(o, + new BigDecimal("-0.99999999999999999999"), 18)), generateBytes(1, 0, 0x9C, 0x58, 0x4C, 0x49, 0x1F, 0xF2)); Assert.assertThrows(IllegalArgumentException.class, @@ -1257,14 +1450,20 @@ public void testReadDecimal128() throws IOException { } } - Assert.assertEquals(BinaryStreamUtils.readDecimal128(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 0), new BigDecimal("-1")); + Assert.assertEquals( + BinaryStreamUtils.readDecimal128(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 0), + new BigDecimal("-1")); Assert.assertEquals(BinaryStreamUtils.readDecimal128( - generateInput(0, 0, 0, 0, 0x40, 0x22, 0x8A, 0x09, 0x7A, 0xC4, 0x86, 0x5A, 0xA8, 0x4C, 0x3B, 0x4B), 38), + generateInput(0, 0, 0, 0, 0x40, 0x22, 0x8A, 0x09, 0x7A, 0xC4, 0x86, 0x5A, 0xA8, 0x4C, + 0x3B, 0x4B), + 38), new BigDecimal("1.00000000000000000000000000000000000000")); Assert.assertEquals( - BinaryStreamUtils.readDecimal128(generateInput(0, 0, 0, 0, 0xC0, 0xDD, 0x75, 0xF6, 0x85, 0x3B, 0x79, - 0xA5, 0x57, 0xB3, 0xC4, 0xB4), 38), + BinaryStreamUtils.readDecimal128( + generateInput(0, 0, 0, 0, 0xC0, 0xDD, 0x75, 0xF6, 0x85, 0x3B, 0x79, + 0xA5, 0x57, 0xB3, 0xC4, 0xB4), + 38), new BigDecimal("-1.00000000000000000000000000000000000000")); } @@ -1280,25 +1479,30 @@ public void testWriteDecimal128() throws IOException { BigDecimal d = new BigDecimal(new BigInteger(BinaryStreamUtils.reverse(bytes)), scale); if (d.toString().replaceAll("[-.]", "").replaceAll("^0+", "").length() > 38) { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, d, scale))); + () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, + d, scale))); } else { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, d, scale)), b); + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDecimal128(o, d, scale)), b); } } } Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, new BigDecimal("-1"), 0)), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, new BigDecimal("0.999999999999999999999999999999999999999999"), 38)), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0x3F, 0x22, 0x8A, 0x09, 0x7A, 0xC4, 0x86, 0x5A, 0xA8, 0x4C, 0x3B, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0x3F, 0x22, 0x8A, 0x09, 0x7A, 0xC4, 0x86, 0x5A, + 0xA8, 0x4C, 0x3B, 0x4B)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, new BigDecimal("-0.999999999999999999999999999999999999999999"), 38)), - generateBytes(1, 0, 0, 0, 0xC0, 0xDD, 0x75, 0xF6, 0x85, 0x3B, 0x79, 0xA5, 0x57, 0xB3, 0xC4, 0xB4)); + generateBytes(1, 0, 0, 0, 0xC0, 0xDD, 0x75, 0xF6, 0x85, 0x3B, 0x79, 0xA5, 0x57, 0xB3, + 0xC4, 0xB4)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal128(o, BigDecimal.ZERO, -1))); @@ -1320,19 +1524,26 @@ public void testReadDecimal256() throws IOException { } } - Assert.assertEquals(BinaryStreamUtils.readDecimal256(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 0), new BigDecimal("-1")); + Assert.assertEquals( + BinaryStreamUtils.readDecimal256(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF), 0), + new BigDecimal("-1")); Assert.assertEquals( BinaryStreamUtils.readDecimal256( - generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0x10, 0x95, 0x71, 0xF1, 0xA5, 0x75, 0x77, 0x79, 0x29, - 0x65, 0xE8, 0xAB, 0xB4, 0x64, 0x07, 0xB5, 0x15, 0x99, 0x11, 0xA7, 0xCC, 0x1B, 0x16), + generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0x10, 0x95, 0x71, 0xF1, 0xA5, + 0x75, 0x77, 0x79, 0x29, + 0x65, 0xE8, 0xAB, 0xB4, 0x64, 0x07, 0xB5, 0x15, 0x99, + 0x11, 0xA7, 0xCC, 0x1B, 0x16), 76), new BigDecimal("1.0000000000000000000000000000000000000000000000000000000000000000000000000000")); Assert.assertEquals( BinaryStreamUtils.readDecimal256( - generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0x6A, 0x8E, 0x0E, 0x5A, 0x8A, 0x88, 0x86, 0xD6, - 0x9A, 0x17, 0x54, 0x4B, 0x9B, 0xF8, 0x4A, 0xEA, 0x66, 0xEE, 0x58, 0x33, 0xE4, 0xE9), + generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0x6A, 0x8E, 0x0E, 0x5A, + 0x8A, 0x88, 0x86, 0xD6, + 0x9A, 0x17, 0x54, 0x4B, 0x9B, 0xF8, 0x4A, 0xEA, 0x66, + 0xEE, 0x58, 0x33, 0xE4, 0xE9), 76), new BigDecimal("-1.0000000000000000000000000000000000000000000000000000000000000000000000000000")); } @@ -1349,32 +1560,40 @@ public void testWriteDecimal256() throws IOException { BigDecimal d = new BigDecimal(new BigInteger(BinaryStreamUtils.reverse(bytes)), scale); if (d.toString().replaceAll("[-.]", "").replaceAll("^0+", "").length() > 76) { Assert.assertThrows(IllegalArgumentException.class, - () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, d, scale))); + () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, + d, scale))); } else { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, d, scale)), b); + Assert.assertEquals(getWrittenBytes( + o -> BinaryStreamUtils.writeDecimal256(o, d, scale)), b); } } } Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, new BigDecimal("-1"), 0)), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, new BigDecimal( "0.9999999999999999999999999999999999999999999999999999999999999999999999999999"), 76)), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x0F, 0x95, 0x71, 0xF1, 0xA5, 0x75, - 0x77, 0x79, 0x29, 0x65, 0xE8, 0xAB, 0xB4, 0x64, 0x07, 0xB5, 0x15, 0x99, 0x11, 0xA7, 0xCC, 0x1B, + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x0F, 0x95, 0x71, + 0xF1, 0xA5, 0x75, + 0x77, 0x79, 0x29, 0x65, 0xE8, 0xAB, 0xB4, 0x64, 0x07, 0xB5, 0x15, 0x99, + 0x11, 0xA7, 0xCC, 0x1B, 0x16)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, new BigDecimal( "-0.9999999999999999999999999999999999999999999999999999999999999999999999999999"), 76)), - generateBytes(1, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0x6A, 0x8E, 0x0E, 0x5A, 0x8A, 0x88, 0x86, 0xD6, 0x9A, - 0x17, 0x54, 0x4B, 0x9B, 0xF8, 0x4A, 0xEA, 0x66, 0xEE, 0x58, 0x33, 0xE4, 0xE9)); + generateBytes(1, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0x6A, 0x8E, 0x0E, 0x5A, 0x8A, 0x88, 0x86, + 0xD6, 0x9A, + 0x17, 0x54, 0x4B, 0x9B, 0xF8, 0x4A, 0xEA, 0x66, 0xEE, 0x58, 0x33, 0xE4, + 0xE9)); Assert.assertThrows(IllegalArgumentException.class, () -> getWrittenBytes(o -> BinaryStreamUtils.writeDecimal256(o, BigDecimal.ZERO, -1))); @@ -1388,12 +1607,15 @@ public void testReadFloat32() throws IOException { for (int i = 0; i < arr.length; i++) { byte[] bytes = arr[i]; Assert.assertEquals(BinaryStreamUtils.readFloat32(generateInput(bytes)), - Float.intBitsToFloat(new BigInteger(BinaryStreamUtils.reverse(bytes)).intValue())); + Float.intBitsToFloat( + new BigInteger(BinaryStreamUtils.reverse(bytes)).intValue())); } // INF and NaN - Assert.assertEquals(BinaryStreamUtils.readFloat32(generateInput(0, 0, 0x80, 0xFF)), Float.NEGATIVE_INFINITY); - Assert.assertEquals(BinaryStreamUtils.readFloat32(generateInput(0, 0, 0x80, 0x7F)), Float.POSITIVE_INFINITY); + Assert.assertEquals(BinaryStreamUtils.readFloat32(generateInput(0, 0, 0x80, 0xFF)), + Float.NEGATIVE_INFINITY); + Assert.assertEquals(BinaryStreamUtils.readFloat32(generateInput(0, 0, 0x80, 0x7F)), + Float.POSITIVE_INFINITY); Assert.assertEquals(BinaryStreamUtils.readFloat32(generateInput(0, 0, 0xC0, 0x7F)), Float.NaN); } @@ -1425,7 +1647,8 @@ public void testReadFloat64() throws IOException { for (int i = 0; i < arr.length; i++) { byte[] bytes = arr[i]; Assert.assertEquals(BinaryStreamUtils.readFloat64(generateInput(bytes)), - Double.longBitsToDouble(new BigInteger(BinaryStreamUtils.reverse(bytes)).longValue())); + Double.longBitsToDouble( + new BigInteger(BinaryStreamUtils.reverse(bytes)).longValue())); } // INF and NaN @@ -1433,7 +1656,8 @@ public void testReadFloat64() throws IOException { Double.NEGATIVE_INFINITY); Assert.assertEquals(BinaryStreamUtils.readFloat64(generateInput(0, 0, 0, 0, 0, 0, 0xF0, 0x7F)), Double.POSITIVE_INFINITY); - Assert.assertEquals(BinaryStreamUtils.readFloat64(generateInput(0, 0, 0, 0, 0, 0, 0xF8, 0x7F)), Double.NaN); + Assert.assertEquals(BinaryStreamUtils.readFloat64(generateInput(0, 0, 0, 0, 0, 0, 0xF8, 0x7F)), + Double.NaN); } @Test(groups = { "unit" }) @@ -1461,15 +1685,18 @@ public void testWriteFloat64() throws IOException { @Test(groups = { "unit" }) public void testReadGeoPoint() throws IOException { Assert.assertEquals( - BinaryStreamUtils.readGeoPoint(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)), + BinaryStreamUtils.readGeoPoint( + generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)), new double[] { 0D, 0D }); Assert.assertEquals( BinaryStreamUtils - .readGeoPoint(generateInput(0, 0, 0, 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF)), + .readGeoPoint(generateInput(0, 0, 0, 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, + 0, 0xF0, 0xBF)), new double[] { -1D, -1D }); Assert.assertEquals( BinaryStreamUtils - .readGeoPoint(generateInput(0, 0, 0, 0, 0, 0, 0xF0, 0x3F, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F)), + .readGeoPoint(generateInput(0, 0, 0, 0, 0, 0, 0xF0, 0x3F, 0, 0, 0, 0, 0, + 0, 0xF0, 0x3F)), new double[] { 1D, 1D }); } @@ -1498,15 +1725,18 @@ public void testReadGeoRing() throws IOException { Assert.assertEquals(ring.length, 1); Assert.assertEquals(ring[0], new double[] { 0D, 0D }); - ring = BinaryStreamUtils.readGeoRing(generateInput(2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF)); + ring = BinaryStreamUtils + .readGeoRing(generateInput(2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF)); Assert.assertEquals(ring.length, 2); Assert.assertEquals(ring[0], new double[] { 0D, 0D }); Assert.assertEquals(ring[1], new double[] { -1D, -1D }); ring = BinaryStreamUtils.readGeoRing( - generateInput(3, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, - 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F)); + generateInput(3, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, + 0xBF, 0, 0, 0, + 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F, 0, 0, 0, 0, 0, 0, + 0xF0, 0x3F)); Assert.assertEquals(ring.length, 3); Assert.assertEquals(ring[0], new double[] { 0D, 0D }); Assert.assertEquals(ring[1], new double[] { -1D, -1D }); @@ -1515,21 +1745,26 @@ public void testReadGeoRing() throws IOException { @Test(groups = { "unit" }) public void testWriteGeoRing() throws IOException { - Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeGeoRing(o, new double[0][])), generateBytes(0)); + Assert.assertEquals(getWrittenBytes(o -> BinaryStreamUtils.writeGeoRing(o, new double[0][])), + generateBytes(0)); Assert.assertEquals( - getWrittenBytes(o -> BinaryStreamUtils.writeGeoRing(o, new double[][] { new double[] { 0D, 0D } })), + getWrittenBytes(o -> BinaryStreamUtils.writeGeoRing(o, + new double[][] { new double[] { 0D, 0D } })), generateBytes(1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeGeoRing(o, new double[][] { new double[] { 0D, 0D }, new double[] { -1D, -1D } })), - generateBytes(2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, + generateBytes(2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, + 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeGeoRing(o, new double[][] { new double[] { 0D, 0D }, new double[] { -1D, -1D }, new double[] { 1D, 1D } })), - generateBytes(3, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, - 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F)); + generateBytes(3, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xF0, + 0xBF, 0, 0, 0, + 0, 0, 0, 0xF0, 0xBF, 0, 0, 0, 0, 0, 0, 0xF0, 0x3F, 0, 0, 0, 0, 0, 0, + 0xF0, 0x3F)); } @Test(groups = { "unit" }) @@ -1544,11 +1779,16 @@ public void testReadGeoMultiPolygon() throws IOException { @Test(groups = { "unit" }) public void testReadInet4Address() throws IOException { - Assert.assertEquals(BinaryStreamUtils.readInet4Address(generateInput(0, 0, 0, 0)).getHostAddress(), "0.0.0.0"); - Assert.assertEquals(BinaryStreamUtils.readInet4Address(generateInput(1, 0, 0, 0)).getHostAddress(), "0.0.0.1"); - Assert.assertEquals(BinaryStreamUtils.readInet4Address(generateInput(0xFF, 0xFF, 0xFF, 0xFF)).getHostAddress(), + Assert.assertEquals(BinaryStreamUtils.readInet4Address(generateInput(0, 0, 0, 0)).getHostAddress(), + "0.0.0.0"); + Assert.assertEquals(BinaryStreamUtils.readInet4Address(generateInput(1, 0, 0, 0)).getHostAddress(), + "0.0.0.1"); + Assert.assertEquals( + BinaryStreamUtils.readInet4Address(generateInput(0xFF, 0xFF, 0xFF, 0xFF)) + .getHostAddress(), "255.255.255.255"); - Assert.assertEquals(BinaryStreamUtils.readInet4Address(generateInput(1, 5, 0xA8, 0xC0)).getHostAddress(), + Assert.assertEquals( + BinaryStreamUtils.readInet4Address(generateInput(1, 5, 0xA8, 0xC0)).getHostAddress(), "192.168.5.1"); } @@ -1556,63 +1796,84 @@ public void testReadInet4Address() throws IOException { public void testWriteInet4Address() throws IOException { Assert.assertEquals( getWrittenBytes( - o -> BinaryStreamUtils.writeInet4Address(o, (Inet4Address) InetAddress.getByName("0.0.0.0"))), + o -> BinaryStreamUtils.writeInet4Address(o, + (Inet4Address) InetAddress.getByName("0.0.0.0"))), generateBytes(0, 0, 0, 0)); Assert.assertEquals( getWrittenBytes( - o -> BinaryStreamUtils.writeInet4Address(o, (Inet4Address) InetAddress.getByName("0.0.0.1"))), + o -> BinaryStreamUtils.writeInet4Address(o, + (Inet4Address) InetAddress.getByName("0.0.0.1"))), generateBytes(1, 0, 0, 0)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeInet4Address(o, (Inet4Address) InetAddress.getByName("255.255.255.255"))), + o -> BinaryStreamUtils.writeInet4Address(o, + (Inet4Address) InetAddress.getByName("255.255.255.255"))), generateBytes(0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeInet4Address(o, (Inet4Address) InetAddress.getByName("192.168.5.1"))), + o -> BinaryStreamUtils.writeInet4Address(o, + (Inet4Address) InetAddress.getByName("192.168.5.1"))), generateBytes(1, 5, 0xA8, 0xC0)); } @Test(groups = { "unit" }) public void testReadInet6Address() throws IOException { Assert.assertEquals(BinaryStreamUtils - .readInet6Address(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)).getHostAddress(), + .readInet6Address(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)) + .getHostAddress(), "0:0:0:0:0:0:0:0"); Assert.assertEquals(BinaryStreamUtils - .readInet6Address(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1)).getHostAddress(), + .readInet6Address(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1)) + .getHostAddress(), "0:0:0:0:0:0:0:1"); Assert.assertEquals( - BinaryStreamUtils.readInet6Address(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)).getHostAddress(), + BinaryStreamUtils + .readInet6Address(generateInput(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)) + .getHostAddress(), "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"); Assert.assertEquals( - BinaryStreamUtils.readInet6Address(generateInput(0x12, 0x34, 0x56, 0x78, 0x90, 0xAB, 0xCD, 0xEF, 0x43, - 0x21, 0x87, 0x65, 0xBA, 0x09, 0xFE, 0xDC)).getHostAddress(), + BinaryStreamUtils + .readInet6Address(generateInput(0x12, 0x34, 0x56, 0x78, 0x90, 0xAB, + 0xCD, 0xEF, 0x43, + 0x21, 0x87, 0x65, 0xBA, 0x09, 0xFE, 0xDC)) + .getHostAddress(), "1234:5678:90ab:cdef:4321:8765:ba09:fedc"); Assert.assertEquals(BinaryStreamUtils - .readInet6Address(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xFF, 0xFF, 0xC0, 0xA8, 5, 1)) + .readInet6Address(generateInput(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xFF, 0xFF, 0xC0, 0xA8, 5, + 1)) .getHostAddress(), "0:0:0:0:0:ffff:c0a8:501"); } @Test(groups = { "unit" }) public void testWriteInet6Address() throws IOException { Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeInet6Address(o, (Inet6Address) InetAddress.getByName("0:0:0:0:0:0:0:0"))), + o -> BinaryStreamUtils.writeInet6Address(o, + (Inet6Address) InetAddress.getByName("0:0:0:0:0:0:0:0"))), generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)); Assert.assertEquals(getWrittenBytes( - o -> BinaryStreamUtils.writeInet6Address(o, (Inet6Address) InetAddress.getByName("0:0:0:0:0:0:0:1"))), + o -> BinaryStreamUtils.writeInet6Address(o, + (Inet6Address) InetAddress.getByName("0:0:0:0:0:0:0:1"))), generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeInet6Address(o, - (Inet6Address) InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"))), - generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + (Inet6Address) InetAddress + .getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"))), + generateBytes(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeInet6Address(o, - (Inet6Address) InetAddress.getByName("1234:5678:90ab:cdef:4321:8765:ba09:fedc"))), - generateBytes(0x12, 0x34, 0x56, 0x78, 0x90, 0xAB, 0xCD, 0xEF, 0x43, 0x21, 0x87, 0x65, 0xBA, 0x09, 0xFE, + (Inet6Address) InetAddress + .getByName("1234:5678:90ab:cdef:4321:8765:ba09:fedc"))), + generateBytes(0x12, 0x34, 0x56, 0x78, 0x90, 0xAB, 0xCD, 0xEF, 0x43, 0x21, 0x87, 0x65, + 0xBA, 0x09, 0xFE, 0xDC)); Assert.assertEquals( getWrittenBytes(o -> BinaryStreamUtils.writeInet6Address(o, Inet6Address.getByAddress(null, - generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xFF, 0xFF, 0xC0, 0xA8, 5, 1), null))), + generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xFF, 0xFF, + 0xC0, 0xA8, 5, 1), + null))), generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xFF, 0xFF, 0xC0, 0xA8, 5, 1)); } @@ -1623,9 +1884,11 @@ public void testReadUuid() throws IOException { byte[] bytes = arr[i]; UUID uuid = BinaryStreamUtils.readUuid(generateInput(bytes)); Assert.assertEquals(uuid.getMostSignificantBits(), - BinaryStreamUtils.readInt64(new ByteArrayInputStream(bytes, 0, Long.BYTES))); + BinaryStreamUtils + .readInt64(ClickHouseInputStream.of(new ByteArrayInputStream(bytes, 0, Long.BYTES)))); Assert.assertEquals(uuid.getLeastSignificantBits(), - BinaryStreamUtils.readInt64(new ByteArrayInputStream(bytes, Long.BYTES, Long.BYTES))); + BinaryStreamUtils.readInt64( + ClickHouseInputStream.of(new ByteArrayInputStream(bytes, Long.BYTES, Long.BYTES)))); } } diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java index 3c574c6d5..610a32e27 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java @@ -4,6 +4,7 @@ import java.util.Map; import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseResponseSummary; import com.clickhouse.client.data.ClickHouseStreamResponse; import com.clickhouse.client.grpc.impl.Progress; @@ -16,7 +17,7 @@ public class ClickHouseGrpcResponse extends ClickHouseStreamResponse { protected ClickHouseGrpcResponse(ClickHouseConfig config, Map settings, ClickHouseStreamObserver observer) throws IOException { - super(config, observer.getInputStream(), settings, null, observer.getSummary()); + super(config, ClickHouseInputStream.of(observer.getInputStream()), settings, null, observer.getSummary()); this.observer = observer; this.result = null; @@ -24,7 +25,8 @@ protected ClickHouseGrpcResponse(ClickHouseConfig config, Map se protected ClickHouseGrpcResponse(ClickHouseConfig config, Map settings, Result result) throws IOException { - super(config, result.getOutput().newInput(), settings, null, new ClickHouseResponseSummary(null, null)); + super(config, ClickHouseInputStream.of(result.getOutput().newInput()), settings, null, + new ClickHouseResponseSummary(null, null)); this.observer = null; this.result = result; diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java index 373119b7f..19d5a03db 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java @@ -1,5 +1,6 @@ package com.clickhouse.client.http; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.Serializable; @@ -10,13 +11,14 @@ import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseFormat; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseResponseSummary; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.config.ClickHouseOption; -public class ClickHouseHttpResponse extends InputStream { +public class ClickHouseHttpResponse extends ClickHouseInputStream { private static long getLongValue(Map map, String key) { String value = map.get(key); if (value != null) { @@ -39,6 +41,8 @@ private static long getLongValue(Map map, String key) { protected final ClickHouseResponseSummary summary; + private boolean closed; + protected ClickHouseConfig getConfig(ClickHouseRequest request) { ClickHouseConfig config = request.getConfig(); if (format != null && format != config.getFormat()) { @@ -86,6 +90,19 @@ public ClickHouseHttpResponse(ClickHouseHttpConnection connection, InputStream i this.timeZone = !ClickHouseChecker.isNullOrEmpty(value) ? TimeZone.getTimeZone(value) : connection.config.getServerTimeZone(); } + + closed = false; + } + + @Override + public byte readByte() throws IOException { + int v = input.read(); + if (v == -1) { + close(); + throw new EOFException(); + } + + return (byte) v; } @Override @@ -98,6 +115,11 @@ public int available() throws IOException { return input.available(); } + @Override + public boolean isClosed() { + return closed; + } + @Override public void close() throws IOException { IOException error = null; @@ -107,6 +129,7 @@ public void close() throws IOException { } catch (IOException e) { error = e; } + closed = true; if (!connection.isReusable()) { try { diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java index d31bd0cb6..648054c69 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java @@ -7,6 +7,8 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.Calendar; +import java.util.Optional; import java.util.TimeZone; import com.clickhouse.client.ClickHouseVersion; @@ -94,8 +96,38 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re */ String getCurrentUser(); - TimeZone getEffectiveTimeZone(); + /** + * Gets default calendar which can be used to create timestamp. + * + * @return non-null calendar + */ + Calendar getDefaultCalendar(); + + /** + * Gets effective time zone. When + * {@link com.clickhouse.client.ClickHouseConfig#isUseServerTimeZone()} returns + * {@code true}, {@link com.clickhouse.client.ClickHouseConfig#getUseTimeZone()} + * will be used as effective time zone, which will be used for reading and + * writing timestamp values. + * + * @return effective time zone + */ + Optional getEffectiveTimeZone(); + /** + * Gets cached value of {@code TimeZone.getDefault()}. + * + * @return non-null cached JVM time zone + */ + TimeZone getJvmTimeZone(); + + /** + * Gets server time zone, which is either same as result of + * {@code select timezone()}, or the overrided value from + * {@link com.clickhouse.client.ClickHouseConfig#getServerTimeZone()}. + * + * @return non-null server time zone + */ TimeZone getServerTimeZone(); ClickHouseVersion getServerVersion(); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java index f51c8eff9..d8687817f 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java @@ -23,6 +23,7 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.util.Calendar; +import java.util.GregorianCalendar; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -49,6 +50,7 @@ public class ClickHouseResultSet extends AbstractResultSet { protected final ClickHouseConfig config; protected final List columns; + protected final Calendar defaultCalendar; protected final TimeZone tsTimeZone; protected final TimeZone dateTimeZone; protected final int maxRows; @@ -62,7 +64,8 @@ public class ClickHouseResultSet extends AbstractResultSet { this.response = response; this.config = null; - this.tsTimeZone = TimeZone.getDefault(); + this.defaultCalendar = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + this.tsTimeZone = null; // TimeZone.getDefault(); this.dateTimeZone = this.tsTimeZone; this.currentRow = null; @@ -95,7 +98,8 @@ public ClickHouseResultSet(String database, String table, ClickHouseStatement st ClickHouseConnection conn = statement.getConnection(); this.config = statement.getConfig(); - this.tsTimeZone = conn.getEffectiveTimeZone(); + this.defaultCalendar = conn.getDefaultCalendar(); + this.tsTimeZone = conn.getEffectiveTimeZone().orElse(null); this.dateTimeZone = this.tsTimeZone; this.currentRow = null; @@ -567,17 +571,10 @@ public Time getTime(int columnIndex, Calendar cal) throws SQLException { // unfortunately java.sql.Time does not support fractional seconds LocalTime lt = value.asTime(); - Time t; - if (cal == null) { - t = Time.valueOf(lt); - } else { - Calendar c = (Calendar) cal.clone(); - c.clear(); - c.set(1970, 0, 1, lt.getHour(), lt.getMinute(), lt.getSecond()); - t = new Time(c.getTimeInMillis()); - } - - return t; + Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); + c.clear(); + c.set(1970, 0, 1, lt.getHour(), lt.getMinute(), lt.getSecond()); + return new Time(c.getTimeInMillis()); } @Override @@ -607,16 +604,11 @@ public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException LocalDateTime dt = tz == null ? value.asDateTime(column.getScale()) : value.asOffsetDateTime(column.getScale()).toLocalDateTime(); - Timestamp timestamp; - if (cal == null) { - timestamp = Timestamp.valueOf(dt); - } else { - Calendar c = (Calendar) cal.clone(); - c.set(dt.getYear(), dt.getMonthValue() - 1, dt.getDayOfMonth(), dt.getHour(), dt.getMinute(), - dt.getSecond()); - timestamp = new Timestamp(c.getTimeInMillis()); - timestamp.setNanos(dt.getNano()); - } + Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); + c.set(dt.getYear(), dt.getMonthValue() - 1, dt.getDayOfMonth(), dt.getHour(), dt.getMinute(), + dt.getSecond()); + Timestamp timestamp = new Timestamp(c.getTimeInMillis()); + timestamp.setNanos(dt.getNano()); return timestamp; } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java index d355d7dce..1a7a9e23a 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java @@ -18,10 +18,13 @@ import java.sql.Savepoint; import java.sql.Struct; import java.util.ArrayList; +import java.util.Calendar; import java.util.Collections; +import java.util.GregorianCalendar; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.TimeZone; import java.util.UUID; @@ -87,8 +90,10 @@ public class ClickHouseConnectionImpl extends Wrapper implements ClickHouseConne private int rsHoldability; private int txIsolation; + private final Optional clientTimeZone; + private final Calendar defaultCalendar; + private final TimeZone jvmTimeZone; private final TimeZone serverTimeZone; - private final TimeZone clientTimeZone; private final ClickHouseVersion serverVersion; private final String user; @@ -163,6 +168,8 @@ public ClickHouseConnectionImpl(String url, Properties properties) throws SQLExc ClickHouseNode node = connInfo.getServer(); log.debug("Target node: %s", node); + jvmTimeZone = TimeZone.getDefault(); + client = ClickHouseClient.builder().options(connInfo.getProperties()) .nodeSelector(ClickHouseNodeSelector.of(node.getProtocol())).build(); clientRequest = client.connect(node); @@ -214,9 +221,16 @@ public ClickHouseConnectionImpl(String url, Properties properties) throws SQLExc this.user = currentUser != null ? currentUser : node.getCredentials(config).getUserName(); this.serverTimeZone = timeZone; - // cannot compare two timezone without giving a specific timestamp - this.clientTimeZone = config.isUseServerTimeZone() ? serverTimeZone - : TimeZone.getTimeZone(config.getUseTimeZone()); + if (config.isUseServerTimeZone()) { + clientTimeZone = Optional.empty(); + // with respect of default locale + defaultCalendar = new GregorianCalendar(); + } else { + clientTimeZone = Optional + .of(ClickHouseChecker.isNullOrBlank(config.getUseTimeZone()) ? TimeZone.getDefault() + : TimeZone.getTimeZone(config.getUseTimeZone())); + defaultCalendar = new GregorianCalendar(clientTimeZone.get()); + } this.serverVersion = version; this.typeMap = new HashMap<>(); this.fakeTransaction = new AtomicReference<>(); @@ -242,7 +256,7 @@ public void setAutoCommit(boolean autoCommit) throws SQLException { if (this.autoCommit = autoCommit) { // commit FakeTransaction tx = fakeTransaction.getAndSet(null); if (tx != null) { - tx.logTransactionDetails(log, "committed"); + tx.logTransactionDetails(log, FakeTransaction.ACTION_COMMITTED); tx.clear(); } } else { // start new transaction @@ -272,9 +286,9 @@ public void commit() throws SQLException { FakeTransaction tx = fakeTransaction.getAndSet(new FakeTransaction()); if (tx == null) { // invalid transaction state - throw new SQLException("Transaction not started", SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { - tx.logTransactionDetails(log, "committed"); + tx.logTransactionDetails(log, FakeTransaction.ACTION_COMMITTED); tx.clear(); } } @@ -292,9 +306,9 @@ public void rollback() throws SQLException { FakeTransaction tx = fakeTransaction.getAndSet(new FakeTransaction()); if (tx == null) { // invalid transaction state - throw new SQLException("Transaction not started", SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { - tx.logTransactionDetails(log, "rolled back"); + tx.logTransactionDetails(log, FakeTransaction.ACTION_ROLLBACK); tx.clear(); } } @@ -312,7 +326,7 @@ public void close() throws SQLException { FakeTransaction tx = fakeTransaction.getAndSet(null); if (tx != null) { - tx.logTransactionDetails(log, "committed"); + tx.logTransactionDetails(log, FakeTransaction.ACTION_COMMITTED); tx.clear(); } } @@ -460,10 +474,10 @@ public void rollback(Savepoint savepoint) throws SQLException { FakeTransaction tx = fakeTransaction.get(); if (tx == null) { // invalid transaction state - throw new SQLException("Transaction not started", SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { FakeSavepoint s = (FakeSavepoint) savepoint; - tx.logSavepointDetails(log, s, "rolled back"); + tx.logSavepointDetails(log, s, FakeTransaction.ACTION_ROLLBACK); tx.toSavepoint(s); } } @@ -487,7 +501,7 @@ public void releaseSavepoint(Savepoint savepoint) throws SQLException { FakeTransaction tx = fakeTransaction.get(); if (tx == null) { // invalid transaction state - throw new SQLException("Transaction not started", SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { FakeSavepoint s = (FakeSavepoint) savepoint; tx.logSavepointDetails(log, s, "released"); @@ -764,10 +778,20 @@ public String getCurrentUser() { } @Override - public TimeZone getEffectiveTimeZone() { + public Calendar getDefaultCalendar() { + return defaultCalendar; + } + + @Override + public Optional getEffectiveTimeZone() { return clientTimeZone; } + @Override + public TimeZone getJvmTimeZone() { + return jvmTimeZone; + } + @Override public TimeZone getServerTimeZone() { return serverTimeZone; diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java index f85124a00..0d5292bef 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java @@ -14,6 +14,11 @@ import com.clickhouse.jdbc.SqlExceptionUtils; public final class FakeTransaction { + static final String ACTION_COMMITTED = "committed"; + static final String ACTION_ROLLBACK = "rolled back"; + + static final String ERROR_TX_NOT_STARTED = "Transaction not started"; + static final int DEFAULT_TX_ISOLATION_LEVEL = Connection.TRANSACTION_READ_UNCOMMITTED; static final class FakeSavepoint implements Savepoint { diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java index ef6f35b98..8e68e7b94 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.sql.Types; import java.time.LocalDateTime; +import java.time.ZoneId; import java.util.ArrayList; import java.util.Calendar; import java.util.GregorianCalendar; @@ -29,6 +30,7 @@ import java.util.List; import java.util.TimeZone; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; @@ -47,6 +49,8 @@ public class SqlBasedPreparedStatement extends ClickHouseStatementImpl implement private static final Logger log = LoggerFactory.getLogger(SqlBasedPreparedStatement.class); private final Calendar defaultCalendar; + private final ZoneId jvmZoneId; + private final JdbcParameterizedQuery preparedQuery; private final ClickHouseValue[] templates; private final String[] values; @@ -57,8 +61,8 @@ protected SqlBasedPreparedStatement(ClickHouseConnection connection, ClickHouseR throws SQLException { super(connection, request, resultSetType, resultSetConcurrency, resultSetHoldability); - defaultCalendar = new GregorianCalendar(); - defaultCalendar.setTimeZone(connection.getEffectiveTimeZone()); + defaultCalendar = connection.getDefaultCalendar(); + jvmZoneId = connection.getJvmTimeZone().toZoneId(); this.preparedQuery = preparedQuery; @@ -268,7 +272,7 @@ public void setAsciiStream(int parameterIndex, InputStream x, int length) throws String s = null; if (x != null) { try { - s = BinaryStreamUtils.readFixedString(x, length, StandardCharsets.US_ASCII); + s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.US_ASCII); } catch (Throwable e) { // IOException and potentially OOM error throw SqlExceptionUtils.clientError(e); } @@ -282,7 +286,7 @@ public void setUnicodeStream(int parameterIndex, InputStream x, int length) thro String s = null; if (x != null) { try { - s = BinaryStreamUtils.readFixedString(x, length, StandardCharsets.UTF_8); + s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.UTF_8); } catch (Throwable e) { // IOException and potentially OOM error throw SqlExceptionUtils.clientError(e); } @@ -465,7 +469,7 @@ public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws S LocalDateTime dt = null; if (cal != null) { - dt = x.toLocalDateTime().atZone(TimeZone.getDefault().toZoneId()) + dt = x.toLocalDateTime().atZone(jvmZoneId) .withZoneSameInstant(cal.getTimeZone().toZoneId()).toLocalDateTime(); } else { dt = x.toLocalDateTime(); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java index 08aa9ddbe..3009088c7 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java @@ -23,16 +23,15 @@ import java.sql.Timestamp; import java.sql.Types; import java.time.LocalDateTime; +import java.time.ZoneId; import java.util.ArrayList; import java.util.Calendar; -import java.util.Collections; -import java.util.GregorianCalendar; import java.util.LinkedList; import java.util.List; -import java.util.TimeZone; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; @@ -50,6 +49,8 @@ public class StreamBasedPreparedStatement extends ClickHouseStatementImpl implem private static final Logger log = LoggerFactory.getLogger(StreamBasedPreparedStatement.class); private final Calendar defaultCalendar; + private final ZoneId jvmZoneId; + private final List columns; private final ClickHouseValue[] values; private final boolean[] flags; @@ -62,8 +63,8 @@ protected StreamBasedPreparedStatement(ClickHouseConnection connection, ClickHou throws SQLException { super(connection, request, resultSetType, resultSetConcurrency, resultSetHoldability); - defaultCalendar = new GregorianCalendar(); - defaultCalendar.setTimeZone(connection.getEffectiveTimeZone()); + defaultCalendar = connection.getDefaultCalendar(); + jvmZoneId = connection.getJvmTimeZone().toZoneId(); this.columns = columns; int size = columns.size(); @@ -231,7 +232,7 @@ public void setAsciiStream(int parameterIndex, InputStream x, int length) throws String s = null; if (x != null) { try { - s = BinaryStreamUtils.readFixedString(x, length, StandardCharsets.US_ASCII); + s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.US_ASCII); } catch (Throwable e) { // IOException and potentially OOM error throw SqlExceptionUtils.clientError(e); } @@ -245,7 +246,7 @@ public void setUnicodeStream(int parameterIndex, InputStream x, int length) thro String s = null; if (x != null) { try { - s = BinaryStreamUtils.readFixedString(x, length, StandardCharsets.UTF_8); + s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.UTF_8); } catch (Throwable e) { // IOException and potentially OOM error throw SqlExceptionUtils.clientError(e); } @@ -432,7 +433,7 @@ public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws S LocalDateTime dt = null; if (cal != null) { - dt = x.toLocalDateTime().atZone(TimeZone.getDefault().toZoneId()) + dt = x.toLocalDateTime().atZone(jvmZoneId) .withZoneSameInstant(cal.getTimeZone().toZoneId()).toLocalDateTime(); } else { dt = x.toLocalDateTime(); diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java index c64ee9f99..905974162 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java @@ -1,9 +1,12 @@ package com.clickhouse.jdbc; +import java.math.BigDecimal; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Statement; import java.util.Arrays; import java.util.Collections; +import java.util.Properties; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.data.ClickHouseSimpleResponse; @@ -12,28 +15,39 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -public class CombinedResultSetTest { +public class CombinedResultSetTest extends JdbcIntegrationTest { @DataProvider(name = "multipleResultSetsProvider") private Object[][] getMultipleResultSets() { return new Object[][] { { new CombinedResultSet(null, new ClickHouseResultSet("", "", ClickHouseSimpleResponse.of(ClickHouseColumn.parse("s String"), - new Object[][] { new Object[] { "a" }, new Object[] { "b" } })), + new Object[][] { new Object[] { "a" }, + new Object[] { "b" } })), new ClickHouseResultSet("", "", - ClickHouseSimpleResponse.of(ClickHouseColumn.parse("s String"), - new Object[][] { new Object[] { "c" }, new Object[] { "d" }, + ClickHouseSimpleResponse.of( + ClickHouseColumn.parse("s String"), + new Object[][] { new Object[] { "c" }, + new Object[] { "d" }, new Object[] { "e" } }))) }, { new CombinedResultSet(Arrays.asList(null, null, new ClickHouseResultSet("", "", ClickHouseSimpleResponse.of( - ClickHouseColumn.parse("s String"), new Object[][] { new Object[] { "a" } })), + ClickHouseColumn.parse("s String"), + new Object[][] { new Object[] { + "a" } })), null, new ClickHouseResultSet("", "", - ClickHouseSimpleResponse.of(ClickHouseColumn.parse("s String"), - new Object[][] { new Object[] { "b" } })), + ClickHouseSimpleResponse.of( + ClickHouseColumn.parse("s String"), + new Object[][] { new Object[] { + "b" } })), new ClickHouseResultSet("", "", - ClickHouseSimpleResponse.of(ClickHouseColumn.parse("s String"), new Object[][] { - new Object[] { "c" }, new Object[] { "d" }, new Object[] { "e" } })))) } }; + ClickHouseSimpleResponse.of( + ClickHouseColumn.parse("s String"), + new Object[][] { + new Object[] { "c" }, + new Object[] { "d" }, + new Object[] { "e" } })))) } }; } @DataProvider(name = "nullOrEmptyResultSetProvider") @@ -51,10 +65,13 @@ private Object[][] getSingleResultSet() { return new Object[][] { { new CombinedResultSet(new ClickHouseResultSet("", "", ClickHouseSimpleResponse.of(ClickHouseColumn.parse("s String"), - new Object[][] { new Object[] { "a" }, new Object[] { "b" } }))) }, + new Object[][] { new Object[] { "a" }, + new Object[] { "b" } }))) }, { new CombinedResultSet(Collections.singleton( - new ClickHouseResultSet("", "", ClickHouseSimpleResponse.of(ClickHouseColumn.parse("s String"), - new Object[][] { new Object[] { "a" }, new Object[] { "b" } })))) } }; + new ClickHouseResultSet("", "", ClickHouseSimpleResponse.of( + ClickHouseColumn.parse("s String"), + new Object[][] { new Object[] { "a" }, + new Object[] { "b" } })))) } }; } @Test(dataProvider = "multipleResultSetsProvider", groups = "unit") @@ -112,4 +129,18 @@ public void testSingleResultSet(CombinedResultSet combined) throws SQLException combined.close(); Assert.assertTrue(combined.isClosed()); } + + @Test(groups = "integration") + public void testBigDecimal() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("select toDecimal64(number / 10, 1) from numbers(10)"); + BigDecimal v = BigDecimal.valueOf(0L).setScale(1); + while (rs.next()) { + Assert.assertEquals(rs.getBigDecimal(1), v); + Assert.assertEquals(rs.getObject(1), v); + v = v.add(new BigDecimal("0.1")); + } + } + } }