Skip to content

Commit badbdcf

Browse files
authored
Merge pull request #2077 from ClickHouse/support_variant_type
Support variant type
2 parents 6788dbc + a34707e commit badbdcf

File tree

18 files changed

+1254
-53
lines changed

18 files changed

+1254
-53
lines changed

clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java

Lines changed: 109 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,13 +37,19 @@
3737

3838
import java.io.Serializable;
3939
import java.lang.reflect.Array;
40+
import java.math.BigInteger;
4041
import java.time.OffsetDateTime;
4142
import java.util.ArrayList;
4243
import java.util.Arrays;
44+
import java.util.Collection;
4345
import java.util.Collections;
46+
import java.util.Comparator;
47+
import java.util.HashMap;
4448
import java.util.LinkedList;
4549
import java.util.List;
50+
import java.util.Map;
4651
import java.util.Objects;
52+
import java.util.Set;
4753
import java.util.TimeZone;
4854

4955
/**
@@ -65,6 +71,7 @@ public final class ClickHouseColumn implements Serializable {
6571
private static final String KEYWORD_OBJECT = ClickHouseDataType.Object.name();
6672
private static final String KEYWORD_MAP = ClickHouseDataType.Map.name();
6773
private static final String KEYWORD_NESTED = ClickHouseDataType.Nested.name();
74+
private static final String KEYWORD_VARIANT = ClickHouseDataType.Variant.name();
6875

6976
private int columnCount;
7077
private int columnIndex;
@@ -92,6 +99,14 @@ public final class ClickHouseColumn implements Serializable {
9299

93100
private ClickHouseValue template;
94101

102+
private Map<Class<?>, Integer> classToVariantOrdNumMap;
103+
104+
private Map<Class<?>, Integer> arrayToVariantOrdNumMap;
105+
106+
private Map<Class<?>, Integer> mapKeyToVariantOrdNumMap;
107+
private Map<Class<?>, Integer> mapValueToVariantOrdNumMap;
108+
109+
95110
private static ClickHouseColumn update(ClickHouseColumn column) {
96111
column.enumConstants = ClickHouseEnum.EMPTY;
97112
int size = column.parameters.size();
@@ -273,6 +288,9 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
273288
case Nothing:
274289
column.template = ClickHouseEmptyValue.INSTANCE;
275290
break;
291+
case Variant:
292+
column.template = ClickHouseTupleValue.of();
293+
break;
276294
default:
277295
break;
278296
}
@@ -398,7 +416,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
398416
fixedLength = false;
399417
estimatedLength++;
400418
} else if (args.startsWith(matchedKeyword = KEYWORD_TUPLE, i)
401-
|| args.startsWith(matchedKeyword = KEYWORD_OBJECT, i)) {
419+
|| args.startsWith(matchedKeyword = KEYWORD_OBJECT, i)
420+
|| args.startsWith(matchedKeyword = KEYWORD_VARIANT, i)) {
402421
int index = args.indexOf('(', i + matchedKeyword.length());
403422
if (index < i) {
404423
throw new IllegalArgumentException(ERROR_MISSING_NESTED_TYPE);
@@ -410,12 +429,22 @@ protected static int readColumn(String args, int startIndex, int len, String nam
410429
if (c == ')') {
411430
break;
412431
} else if (c != ',' && !Character.isWhitespace(c)) {
432+
String columnName = "";
413433
i = readColumn(args, i, endIndex, "", nestedColumns);
414434
}
415435
}
416436
if (nestedColumns.isEmpty()) {
417437
throw new IllegalArgumentException("Tuple should have at least one nested column");
418438
}
439+
440+
List<ClickHouseDataType> variantDataTypes = new ArrayList<>();
441+
if (matchedKeyword.equals(KEYWORD_VARIANT)) {
442+
nestedColumns.sort(Comparator.comparing(o -> o.getDataType().name()));
443+
nestedColumns.forEach(c -> {
444+
c.columnName = "v." + c.getDataType().name();
445+
variantDataTypes.add(c.dataType);
446+
});
447+
}
419448
column = new ClickHouseColumn(ClickHouseDataType.valueOf(matchedKeyword), name,
420449
args.substring(startIndex, endIndex + 1), nullable, lowCardinality, null, nestedColumns);
421450
for (ClickHouseColumn n : nestedColumns) {
@@ -424,6 +453,39 @@ protected static int readColumn(String args, int startIndex, int len, String nam
424453
fixedLength = false;
425454
}
426455
}
456+
column.classToVariantOrdNumMap = ClickHouseDataType.buildVariantMapping(variantDataTypes);
457+
458+
for (int ordNum = 0; ordNum < nestedColumns.size(); ordNum++) {
459+
ClickHouseColumn nestedColumn = nestedColumns.get(ordNum);
460+
if (nestedColumn.getDataType() == ClickHouseDataType.Array) {
461+
Set<Class<?>> classSet = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nestedColumn.arrayBaseColumn.dataType);
462+
if (classSet != null) {
463+
if (column.arrayToVariantOrdNumMap == null) {
464+
column.arrayToVariantOrdNumMap = new HashMap<>();
465+
}
466+
for (Class<?> c : classSet) {
467+
column.arrayToVariantOrdNumMap.put(c, ordNum);
468+
}
469+
}
470+
} else if (nestedColumn.getDataType() == ClickHouseDataType.Map) {
471+
Set<Class<?>> keyClassSet = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nestedColumn.getKeyInfo().getDataType());
472+
Set<Class<?>> valueClassSet = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nestedColumn.getValueInfo().getDataType());
473+
if (keyClassSet != null && valueClassSet != null) {
474+
if (column.mapKeyToVariantOrdNumMap == null) {
475+
column.mapKeyToVariantOrdNumMap = new HashMap<>();
476+
}
477+
if (column.mapValueToVariantOrdNumMap == null) {
478+
column.mapValueToVariantOrdNumMap = new HashMap<>();
479+
}
480+
for (Class<?> c : keyClassSet) {
481+
column.mapKeyToVariantOrdNumMap.put(c, ordNum);
482+
}
483+
for (Class<?> c : valueClassSet) {
484+
column.mapValueToVariantOrdNumMap.put(c, ordNum);
485+
}
486+
}
487+
}
488+
}
427489
}
428490

429491
if (column == null) {
@@ -627,6 +689,52 @@ public boolean isAggregateFunction() {
627689

628690
}
629691

692+
public int getVariantOrdNum(Object value) {
693+
if (value != null && value.getClass().isArray()) {
694+
// TODO: add cache by value class
695+
Class<?> c = value.getClass();
696+
while (c.isArray()) {
697+
c = c.getComponentType();
698+
}
699+
return arrayToVariantOrdNumMap.getOrDefault(c, -1);
700+
} else if (value != null && value instanceof List<?>) {
701+
// TODO: add cache by instance of the list
702+
Object tmpV = ((List) value).get(0);
703+
Class<?> valueClass = tmpV.getClass();
704+
while (tmpV instanceof List<?>) {
705+
tmpV = ((List) tmpV).get(0);
706+
valueClass = tmpV.getClass();
707+
}
708+
return arrayToVariantOrdNumMap.getOrDefault(valueClass, -1);
709+
} else if (value != null && value instanceof Map<?,?>) {
710+
// TODO: add cache by instance of map
711+
Map<?, ?> map = (Map<?, ?>) value;
712+
if (!map.isEmpty()) {
713+
for (Map.Entry<?, ?> e : map.entrySet()) {
714+
if (e.getValue() != null) {
715+
int keyOrdNum = mapKeyToVariantOrdNumMap.getOrDefault(e.getKey().getClass(), -1);
716+
int valueOrdNum = mapValueToVariantOrdNumMap.getOrDefault(e.getValue().getClass(), -1);
717+
718+
if (keyOrdNum == valueOrdNum) {
719+
return valueOrdNum; // exact match
720+
} else if (keyOrdNum != -1 && valueOrdNum != -1) {
721+
if (ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nested.get(keyOrdNum).getValueInfo().getDataType()).contains(e.getValue().getClass())){
722+
return keyOrdNum; // can write to map found by key class because values are compatible
723+
} else {
724+
return valueOrdNum;
725+
}
726+
}
727+
728+
break;
729+
}
730+
}
731+
}
732+
return -1;
733+
} else {
734+
return classToVariantOrdNumMap.getOrDefault(value.getClass(), -1);
735+
}
736+
}
737+
630738
public boolean isArray() {
631739
return dataType == ClickHouseDataType.Array;
632740
}

clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java

Lines changed: 118 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,22 +1,35 @@
11
package com.clickhouse.data;
22

3+
import java.lang.reflect.Array;
34
import java.math.BigDecimal;
45
import java.math.BigInteger;
56
import java.net.Inet4Address;
67
import java.net.Inet6Address;
78
import java.time.LocalDate;
89
import java.time.LocalDateTime;
10+
import java.time.ZonedDateTime;
11+
import java.util.ArrayList;
912
import java.util.Arrays;
13+
import java.util.Collection;
1014
import java.util.Collections;
15+
import java.util.Comparator;
16+
import java.util.EnumSet;
1117
import java.util.HashMap;
18+
import java.util.HashSet;
1219
import java.util.LinkedList;
1320
import java.util.List;
1421
import java.util.Locale;
1522
import java.util.Map;
1623
import java.util.Set;
24+
import java.util.TreeMap;
1725
import java.util.TreeSet;
1826
import java.util.UUID;
27+
import java.util.stream.Collectors;
1928

29+
import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
30+
import com.clickhouse.data.value.ClickHouseGeoPointValue;
31+
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
32+
import com.clickhouse.data.value.ClickHouseGeoRingValue;
2033
import com.clickhouse.data.value.UnsignedByte;
2134
import com.clickhouse.data.value.UnsignedInteger;
2235
import com.clickhouse.data.value.UnsignedLong;
@@ -101,7 +114,111 @@ public enum ClickHouseDataType {
101114
Nothing(Object.class, false, true, false, 0, 0, 0, 0, 0, true),
102115
SimpleAggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, false),
103116
// implementation-defined intermediate state
104-
AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true);
117+
AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true),
118+
Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true),
119+
120+
;
121+
122+
public static final List<ClickHouseDataType> ORDERED_BY_RANGE_INT_TYPES =
123+
Collections.unmodifiableList(Arrays.asList(
124+
Int8, UInt8, Int16, UInt16, Int32, UInt32, Int64, UInt64, Int128, UInt128, Int256, UInt256
125+
));
126+
127+
public static final List<ClickHouseDataType> ORDERED_BY_RANGE_DECIMAL_TYPES =
128+
Collections.unmodifiableList(Arrays.asList(
129+
Float32, Float64, Decimal32, Decimal64, Decimal128, Decimal256, Decimal
130+
));
131+
132+
public static Map<Class<?>, Integer> buildVariantMapping(List<ClickHouseDataType> variantDataTypes) {
133+
Map<Class<?>, Integer> variantMapping = new HashMap<>();
134+
135+
TreeMap<ClickHouseDataType, Integer> intTypesMappings = new TreeMap<>(Comparator.comparingInt(ORDERED_BY_RANGE_INT_TYPES::indexOf));
136+
TreeMap<ClickHouseDataType, Integer> decTypesMappings = new TreeMap<>(Comparator.comparingInt(ORDERED_BY_RANGE_DECIMAL_TYPES::indexOf));
137+
138+
for (int ordNum = 0; ordNum < variantDataTypes.size(); ordNum++) {
139+
ClickHouseDataType dataType = variantDataTypes.get(ordNum);
140+
Set<Class<?>> classSet = DATA_TYPE_TO_CLASS.get(dataType);
141+
142+
final int finalOrdNum = ordNum;
143+
if (classSet != null) {
144+
if (ORDERED_BY_RANGE_INT_TYPES.contains(dataType)) {
145+
intTypesMappings.put(dataType, ordNum);
146+
} else if (ORDERED_BY_RANGE_DECIMAL_TYPES.contains(dataType)) {
147+
decTypesMappings.put(dataType, ordNum);
148+
} else {
149+
classSet.forEach(c -> variantMapping.put(c, finalOrdNum));
150+
}
151+
}
152+
}
153+
154+
// add integers
155+
for (java.util.Map.Entry<ClickHouseDataType, Integer> entry : intTypesMappings.entrySet()) {
156+
DATA_TYPE_TO_CLASS.get(entry.getKey()).forEach(c -> variantMapping.put(c, entry.getValue()));
157+
}
158+
// add decimals
159+
for (java.util.Map.Entry<ClickHouseDataType, Integer> entry : decTypesMappings.entrySet()) {
160+
DATA_TYPE_TO_CLASS.get(entry.getKey()).forEach(c -> variantMapping.put(c, entry.getValue()));
161+
}
162+
163+
return variantMapping;
164+
}
165+
166+
static final Map<ClickHouseDataType, Set<Class<?>>> DATA_TYPE_TO_CLASS = dataTypeClassMap();
167+
static Map<ClickHouseDataType, Set<Class<?>>> dataTypeClassMap() {
168+
Map<ClickHouseDataType, Set<Class<?>>> map = new HashMap<>();
169+
170+
// We allow to write short to UInt8 even it may not fit. It is done because we have to allow users to utilize UInt* data types.
171+
List<Class<?>> allNumberClassesOrderedBySize = Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class);
172+
Set<Class<?>> setOfAllNumberClasses = Collections.unmodifiableSet(new HashSet<>(allNumberClassesOrderedBySize));
173+
map.put(UInt256, setOfAllNumberClasses);
174+
map.put(Int256, setOfAllNumberClasses);
175+
map.put(UInt128, setOfAllNumberClasses);
176+
map.put(Int128, setOfAllNumberClasses);
177+
map.put(UInt64, setOfAllNumberClasses);
178+
179+
map.put(Int64, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class));
180+
map.put(UInt32, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class ));
181+
map.put(Int32, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class));
182+
map.put(UInt16, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class));
183+
map.put(Int16, setOf(byte.class, Byte.class, short.class, Short.class));
184+
map.put(UInt8, setOf(byte.class, Byte.class, short.class, Short.class));
185+
map.put(Int8, setOf(byte.class, Byte.class));
186+
187+
map.put(Bool, setOf(boolean.class, Boolean.class));
188+
map.put(String, setOf(String.class));
189+
map.put(Float64, setOf(float.class, Float.class, double.class, Double.class));
190+
map.put(Float32, setOf(float.class, Float.class));
191+
map.put(Decimal, setOf(float.class, Float.class, double.class, Double.class, BigDecimal.class));
192+
map.put(Decimal256, setOf(float.class, Float.class, double.class, Double.class, BigDecimal.class));
193+
map.put(Decimal128, setOf(float.class, Float.class, double.class, Double.class, BigDecimal.class));
194+
map.put(Decimal64, setOf(float.class, Float.class, double.class, Double.class));
195+
map.put(Decimal32, setOf(float.class, Float.class));
196+
197+
map.put(IPv4, setOf(Inet4Address.class));
198+
map.put(IPv6, setOf(Inet6Address.class));
199+
map.put(UUID, setOf(java.util.UUID.class));
200+
201+
map.put(Point, setOf(double[].class, ClickHouseGeoPointValue.class));
202+
map.put(Ring, setOf(double[][].class, ClickHouseGeoRingValue.class));
203+
map.put(Polygon, setOf(double[][][].class, ClickHouseGeoPolygonValue.class));
204+
map.put(MultiPolygon, setOf(double[][][][].class, ClickHouseGeoMultiPolygonValue.class));
205+
206+
map.put(Date, setOf(LocalDateTime.class, LocalDate.class, ZonedDateTime.class));
207+
map.put(Date32, setOf(LocalDateTime.class, LocalDate.class, ZonedDateTime.class));
208+
map.put(DateTime64, setOf(LocalDateTime.class, ZonedDateTime.class));
209+
map.put(DateTime32, setOf(LocalDateTime.class, ZonedDateTime.class));
210+
map.put(DateTime, setOf(LocalDateTime.class, ZonedDateTime.class));
211+
212+
map.put(Enum8, setOf(java.lang.String.class,byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class));
213+
map.put(Enum16, setOf(java.lang.String.class,byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class));
214+
map.put(Array, setOf(List.class, Object[].class, byte[].class, short[].class, int[].class, long[].class, boolean[].class));
215+
return map;
216+
}
217+
218+
private static Set<Class<?>> setOf(Class<?>... args) {
219+
return Collections.unmodifiableSet(new HashSet<>(Arrays.stream(args).collect(Collectors.toList())));
220+
}
221+
105222

106223
/**
107224
* Immutable set(sorted) for all aliases.

0 commit comments

Comments
 (0)