Skip to content
Merged
Original file line number Diff line number Diff line change
@@ -1,5 +1,14 @@
package com.clickhouse.data;

import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
import com.clickhouse.data.value.ClickHouseGeoPointValue;
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
import com.clickhouse.data.value.ClickHouseGeoRingValue;
import com.clickhouse.data.value.UnsignedByte;
import com.clickhouse.data.value.UnsignedInteger;
import com.clickhouse.data.value.UnsignedLong;
import com.clickhouse.data.value.UnsignedShort;

import java.math.BigDecimal;
import java.math.BigInteger;
import java.net.Inet4Address;
Expand All @@ -9,7 +18,6 @@
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.Period;
import java.time.ZonedDateTime;
import java.time.temporal.ChronoUnit;
Expand All @@ -29,15 +37,6 @@
import java.util.UUID;
import java.util.stream.Collectors;

import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
import com.clickhouse.data.value.ClickHouseGeoPointValue;
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
import com.clickhouse.data.value.ClickHouseGeoRingValue;
import com.clickhouse.data.value.UnsignedByte;
import com.clickhouse.data.value.UnsignedInteger;
import com.clickhouse.data.value.UnsignedLong;
import com.clickhouse.data.value.UnsignedShort;

/**
* Basic ClickHouse data types.
*
Expand Down Expand Up @@ -101,12 +100,12 @@ public enum ClickHouseDataType implements SQLType {
IPv4(Inet4Address.class, false, true, false, 4, 10, 0, 0, 0, false, 0x28, "INET4"),
IPv6(Inet6Address.class, false, true, false, 16, 39, 0, 0, 0, false, 0x29, "INET6"),
UUID(UUID.class, false, true, false, 16, 69, 0, 0, 0, false, 0x1D),
Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true, 0x2C), // same as Tuple(Float64, Float64)
Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring)
MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Polygon)
Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point)
LineString( Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point)
MultiLineString(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring)
Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true), // same as Tuple(Float64, Float64)
Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring)
MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Polygon)
Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point)
LineString( Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point)
MultiLineString(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring)

JSON(Object.class, false, false, false, 0, 0, 0, 0, 0, true, 0x30),
@Deprecated
Expand Down Expand Up @@ -372,11 +371,11 @@ public byte getTag() {
allAliases = Collections.unmodifiableSet(set);
name2type = Collections.unmodifiableMap(map);

Map<Byte, ClickHouseDataType> tmpbinTag2Type = new HashMap<>();
Map<Byte, ClickHouseDataType> tmpBinTag2Type = new HashMap<>();
for (ClickHouseDataType type : ClickHouseDataType.values()) {
tmpbinTag2Type.put((byte) type.getBinTag(), type);
tmpBinTag2Type.put(type.getBinTag(), type);
}
binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type);
binTag2Type = Collections.unmodifiableMap(tmpBinTag2Type);

Map<Byte, ClickHouseDataType> tmpIntervalKind2Type = new HashMap<>();
Map<ClickHouseDataType, ClickHouseDataType.IntervalKind > tmpIntervalType2Kind = new HashMap<>();
Expand Down
2 changes: 1 addition & 1 deletion client-v2/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<scope>test</scope>
<version>2.17.2</version>
<version>${jackson.version}</version>
</dependency>
<dependency>
<groupId>${project.parent.groupId}</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,15 +25,11 @@
import java.time.ZonedDateTime;
import java.time.temporal.TemporalAmount;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.TimeZone;
import java.util.UUID;

Expand All @@ -58,6 +54,10 @@ public class BinaryStreamReader {

private final Class<?> arrayDefaultTypeHint;

private static final int SB_INIT_SIZE = 100;

private ClickHouseColumn lastDataColumn = null;

/**
* Createa a BinaryStreamReader instance that will use the provided buffer allocator.
*
Expand Down Expand Up @@ -110,6 +110,7 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
}

ClickHouseColumn actualColumn = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column;
lastDataColumn = actualColumn;
ClickHouseDataType dataType = actualColumn.getDataType();
int precision = actualColumn.getPrecision();
int scale = actualColumn.getScale();
Expand Down Expand Up @@ -1149,73 +1150,176 @@ public byte[] allocate(int size) {
}
}

private static final Set<Byte> DECIMAL_TAGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
ClickHouseDataType.Decimal.getBinTag(),
ClickHouseDataType.Decimal32.getBinTag(),
ClickHouseDataType.Decimal64.getBinTag(),
ClickHouseDataType.Decimal128.getBinTag(),
ClickHouseDataType.Decimal256.getBinTag()
)));

private ClickHouseColumn readDynamicData() throws IOException {
byte tag = readByte();

ClickHouseDataType type;
if (tag == ClickHouseDataType.INTERVAL_BIN_TAG) {
byte intervalKind = readByte();
type = ClickHouseDataType.intervalKind2Type.get(intervalKind);
if (type == null) {
throw new ClientException("Unsupported interval kind: " + intervalKind);
ClickHouseDataType type = ClickHouseDataType.binTag2Type.get(tag);
if (type == null) {
if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) {
// without timezone
byte scale = readByte();
return ClickHouseColumn.of("v", "DateTime64(" + scale + ")");
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
String typeName = readString(input);
return ClickHouseColumn.of("v", typeName);
} else if (tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG || tag == ClickHouseDataType.TUPLE_WITHOUT_NAMES_BIN_TAG) {
int size = readVarInt(input);
StringBuilder typeNameBuilder = new StringBuilder(SB_INIT_SIZE);
typeNameBuilder.append("Tuple(");
final boolean readName = tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG;
for (int i = 0; i < size; i++) {
if (readName) {
String name = readString(input);
typeNameBuilder.append(name).append(' ');
}
ClickHouseColumn column = readDynamicData();
typeNameBuilder.append(column.getOriginalTypeName()).append(',');
}
typeNameBuilder.setLength(typeNameBuilder.length() - 1);
typeNameBuilder.append(")");
return ClickHouseColumn.of("v", typeNameBuilder.toString());
} else {
throw new ClientException("Unsupported data type with tag " + tag);
}
return ClickHouseColumn.of("v", type, false, 0, 0);
} else if (tag == ClickHouseDataType.DateTime32.getBinTag()) {
byte scale = readByte();
return ClickHouseColumn.of("v", "DateTime32(" + scale + ")");
} else if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { // without timezone
byte scale = readByte();
return ClickHouseColumn.of("v", "DateTime64(" + scale +")");
} else if (tag == ClickHouseDataType.DateTime64.getBinTag()) {
byte scale = readByte();
String timezone = readString(input);
return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")");
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
String typeName = readString(input);
return ClickHouseColumn.of("v", typeName);
} else if (DECIMAL_TAGS.contains(tag)) {
int precision = readByte();
int scale = readByte();
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
} else if (tag == ClickHouseDataType.Array.getBinTag()) {
ClickHouseColumn elementColumn = readDynamicData();
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
} else if (tag == ClickHouseDataType.Map.getBinTag()) {
ClickHouseColumn keyInfo = readDynamicData();
ClickHouseColumn valueInfo = readDynamicData();
return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")");
} else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) {
int constants = readVarInt(input);
int[] values = new int[constants];
String[] names = new String[constants];
ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8;
for (int i = 0; i < constants; i++) {
names[i] = readString(input);
if (enumType == ClickHouseDataType.Enum8) {
values[i] = readUnsignedByte();
} else {
values[i] = readUnsignedShortLE();
}
switch (type) {
case Array: {
ClickHouseColumn elementColumn = readDynamicData();
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
}
case DateTime32: {
String timezone = readString(input);
return ClickHouseColumn.of("v", "DateTime32(" + timezone + ")");
}
case DateTime64: {
byte scale = readByte();
String timezone = readString(input);
return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")");
}
case Decimal:
case Decimal32:
case Decimal64:
case Decimal128:
case Decimal256: {
int precision = readByte();
int scale = readByte();
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
}
case Dynamic: {
int maxTypes = readVarInt(input);
return ClickHouseColumn.of("v", "Dynamic(" + maxTypes + ")");
}
case Enum:
case Enum8:
case Enum16: {
int constants = readVarInt(input);
int[] values = new int[constants];
String[] names = new String[constants];
ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8;
for (int i = 0; i < constants; i++) {
names[i] = readString(input);
if (enumType == ClickHouseDataType.Enum8) {
values[i] = readUnsignedByte();
} else {
values[i] = readUnsignedShortLE();
}
}
return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(),
new ClickHouseEnum(names, values));
}
return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(),
new ClickHouseEnum(names, values));
} else if (tag == ClickHouseDataType.NULLABLE_BIN_TAG) {
ClickHouseColumn column = readDynamicData();
return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")");
} else {
type = ClickHouseDataType.binTag2Type.get(tag);
if (type == null) {
throw new ClientException("Unsupported data type with tag " + tag);
case FixedString: {
int length = readVarInt(input);
return ClickHouseColumn.of("v", "FixedString(" + length + ")");
}
return ClickHouseColumn.of("v", type, false, 0, 0);
case IntervalHour:
case IntervalMinute:
case IntervalSecond:
case IntervalDay:
case IntervalMonth:
case IntervalMicrosecond:
case IntervalMillisecond:
case IntervalNanosecond:
case IntervalQuarter:
case IntervalYear:
case IntervalWeek: {
byte intervalKind = readByte();
type = ClickHouseDataType.intervalKind2Type.get(intervalKind);
if (type == null) {
throw new ClientException("Unsupported interval kind: " + intervalKind);
}
return ClickHouseColumn.of("v", type, false, 0, 0);
}
case JSON: {
byte serializationVersion = readByte();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

serializationVersion not in use? Seems to be redundant, or any other plans?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I need to figure out. but seems not in use.

int maxDynamicPaths = readVarInt(input);
byte maxDynamicTypes = readByte();
int numberOfTypedPaths = readVarInt(input);
StringBuilder typeDef = new StringBuilder(SB_INIT_SIZE);
typeDef.append("JSON(max_dynamic_paths=").append(maxDynamicPaths).append(",max_dynamic_types=").append(maxDynamicTypes).append(",");
for (int i = 0; i < numberOfTypedPaths; i++) {
typeDef.append(readString(input)).append(' '); // path
ClickHouseColumn column = readDynamicData();
typeDef.append(column.getOriginalTypeName()).append(',');
}
int numberOfSkipPaths = readVarInt(input);
for (int i = 0; i < numberOfSkipPaths; i++) {
typeDef.append(readString(input)).append(',');
}
int numberOfPathRegexp = readVarInt(input);
for (int i = 0; i < numberOfPathRegexp; i++) {
typeDef.append(readString(input)).append(',');
}
typeDef.setLength(typeDef.length() - 1);
typeDef.append(')');
return ClickHouseColumn.of("v", typeDef.toString());
}
case LowCardinality: {
ClickHouseColumn column = readDynamicData();
return ClickHouseColumn.of("v", "LowCardinality(" + column.getOriginalTypeName() + ")");
}
case Map: {
ClickHouseColumn keyInfo = readDynamicData();
ClickHouseColumn valueInfo = readDynamicData();
return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")");
}
case Nested: {
int size = readVarInt(input);
StringBuilder nested = new StringBuilder(SB_INIT_SIZE);
nested.append("Nested(");
for (int i = 0; i < size; i++) {
String name = readString(input);
nested.append(name).append(',');
}
nested.setLength(nested.length() - 1);
nested.append(')');
return ClickHouseColumn.of("v", nested.toString());
}
case Nullable: {
ClickHouseColumn column = readDynamicData();
return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")");
}
case Time64: {
byte precision = readByte();
return ClickHouseColumn.of("v", "Time64(" + precision + ")");
}
case Variant: {
int variants = readVarInt(input);
StringBuilder variant = new StringBuilder(SB_INIT_SIZE);
variant.append("Variant(");
for (int i = 0; i < variants; i++) {
ClickHouseColumn column = readDynamicData();
variant.append(column.getOriginalTypeName()).append(',');
}
variant.setLength(variant.length() - 1);
variant.append(")");
return ClickHouseColumn.of("v", "Variant(" + variant + ")");
}
case AggregateFunction:
throw new ClientException("Aggregate functions are not supported yet");
case BFloat16:
throw new ClientException("BFloat16 is not supported yet");
default:
return ClickHouseColumn.of("v", type, false, 0, 0);
}
}

Expand All @@ -1235,6 +1339,9 @@ private Map<String, Object> readJsonData(InputStream input, ClickHouseColumn col
ClickHouseColumn dataColumn = predefinedColumns == null? JSON_PLACEHOLDER_COL :
predefinedColumns.getOrDefault(path, JSON_PLACEHOLDER_COL);
Object value = readValue(dataColumn);
if (value == null && (lastDataColumn != null && lastDataColumn.getDataType() == ClickHouseDataType.Nothing) ) {
continue;
}
obj.put(path, value);
}
return obj;
Expand Down
Loading
Loading