Skip to content

Commit 2e84b5a

Browse files
authored
Add support for complex union types and maps (#98)
* Add support for complex union types and maps * Add AvroConverter changes * Add tests
1 parent 00e7864 commit 2e84b5a

File tree

7 files changed

+118
-40
lines changed

7 files changed

+118
-40
lines changed

gradle/libs.versions.toml

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1,9 +1,9 @@
11
[libraries]
22
assertj = "org.assertj:assertj-core:3.12.0"
33
avro = "org.apache.avro:avro:1.10.2"
4-
calcite-avatica = "org.apache.calcite.avatica:avatica:1.23.0"
5-
calcite-core = "org.apache.calcite:calcite-core:1.34.0"
6-
calcite-server = "org.apache.calcite:calcite-server:1.34.0"
4+
calcite-avatica = "org.apache.calcite.avatica:avatica:1.25.0"
5+
calcite-core = "org.apache.calcite:calcite-core:1.38.0"
6+
calcite-server = "org.apache.calcite:calcite-server:1.38.0"
77
commons-cli = "commons-cli:commons-cli:1.4"
88
flink-clients = "org.apache.flink:flink-clients:1.18.1"
99
flink-connector-base = "org.apache.flink:flink-connector-base:1.18.1"
@@ -19,16 +19,15 @@ flink-table-common = "org.apache.flink:flink-table-common:1.18.1"
1919
flink-table-planner = "org.apache.flink:flink-table-planner_2.12:1.18.1"
2020
flink-table-runtime = "org.apache.flink:flink-table-runtime:1.18.1"
2121
gson = "com.google.code.gson:gson:2.9.0"
22-
jackson = "com.fasterxml.jackson.core:jackson-core:2.14.1"
23-
jackson-dataformat-yaml = "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.14.1"
22+
jackson = "com.fasterxml.jackson.core:jackson-core:2.15.0"
23+
jackson-dataformat-yaml = "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.15.0"
2424
javax-annotation-api = "javax.annotation:javax.annotation-api:1.3.2"
2525
junit = "junit:junit:4.12"
2626
kafka-clients = "org.apache.kafka:kafka-clients:3.2.0"
27-
kubernetes-client = "io.kubernetes:client-java:16.0.2"
28-
kubernetes-extended-client = "io.kubernetes:client-java-extended:16.0.2"
27+
kubernetes-client = "io.kubernetes:client-java:18.0.0"
28+
kubernetes-extended-client = "io.kubernetes:client-java-extended:18.0.0"
2929
slf4j-simple = "org.slf4j:slf4j-simple:1.7.30"
3030
slf4j-api = "org.slf4j:slf4j-api:1.7.30"
31-
snakeyaml = "org.yaml:snakeyaml:1.33"
3231
sqlline = "sqlline:sqlline:1.12.0"
3332
quidem = "net.hydromatic:quidem:0.11"
3433
venice = "com.linkedin.venice:venice-common:0.4.376"

hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/AvroConverter.java

Lines changed: 20 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,8 @@ public static Schema avro(String namespace, String name, RelDataType dataType) {
3333
switch (dataType.getSqlTypeName()) {
3434
case INTEGER:
3535
return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable());
36+
case SMALLINT:
37+
return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable());
3638
case BIGINT:
3739
return createAvroTypeWithNullability(Schema.Type.LONG, dataType.isNullable());
3840
case VARCHAR:
@@ -48,10 +50,9 @@ public static Schema avro(String namespace, String name, RelDataType dataType) {
4850
case ARRAY:
4951
return createAvroSchemaWithNullability(Schema.createArray(avro(null, null, dataType.getComponentType())),
5052
dataType.isNullable());
51-
// TODO support map types
52-
// Appears to require a Calcite version bump
53-
// case MAP:
54-
// return createAvroSchemaWithNullability(Schema.createMap(avroPrimitive(dataType.getValueType())), dataType.isNullable());
53+
case MAP:
54+
return createAvroSchemaWithNullability(Schema.createMap(avro(null, null, dataType.getValueType())),
55+
dataType.isNullable());
5556
case UNKNOWN:
5657
case NULL:
5758
return Schema.createUnion(Schema.create(Schema.Type.NULL));
@@ -90,12 +91,11 @@ public static RelDataType rel(Schema schema, RelDataTypeFactory typeFactory, boo
9091
RelDataType unknown = typeFactory.createUnknownType();
9192
switch (schema.getType()) {
9293
case RECORD:
93-
return typeFactory.createStructType(schema.getFields()
94-
.stream()
94+
return typeFactory.createTypeWithNullability(typeFactory.createStructType(schema.getFields().stream()
9595
.map(x -> new AbstractMap.SimpleEntry<>(x.name(), rel(x.schema(), typeFactory, nullable)))
9696
.filter(x -> x.getValue().getSqlTypeName() != SqlTypeName.NULL)
9797
.filter(x -> x.getValue().getSqlTypeName() != unknown.getSqlTypeName())
98-
.collect(Collectors.toList()));
98+
.collect(Collectors.toList())), nullable);
9999
case INT:
100100
return createRelType(typeFactory, SqlTypeName.INTEGER, nullable);
101101
case LONG:
@@ -113,21 +113,25 @@ public static RelDataType rel(Schema schema, RelDataTypeFactory typeFactory, boo
113113
case BOOLEAN:
114114
return createRelType(typeFactory, SqlTypeName.BOOLEAN, nullable);
115115
case ARRAY:
116-
return typeFactory.createArrayType(rel(schema.getElementType(), typeFactory, true), -1);
117-
// TODO support map types
118-
// Appears to require a Calcite version bump
119-
// case MAP:
120-
// return typeFactory.createMapType(typeFactory.createSqlType(SqlTypeName.VARCHAR), rel(schema.getValueType(), typeFactory, nullable))
116+
return typeFactory.createTypeWithNullability(
117+
typeFactory.createArrayType(rel(schema.getElementType(), typeFactory, true), -1), nullable);
118+
case MAP:
119+
return typeFactory.createTypeWithNullability(
120+
typeFactory.createMapType(typeFactory.createSqlType(SqlTypeName.VARCHAR), rel(schema.getValueType(), typeFactory, nullable)), nullable);
121121
case UNION:
122+
boolean isNullable = schema.isNullable();
122123
if (schema.isNullable() && schema.getTypes().size() == 2) {
123124
Schema innerType = schema.getTypes().stream().filter(x -> x.getType() != Schema.Type.NULL).findFirst().get();
124125
return typeFactory.createTypeWithNullability(rel(innerType, typeFactory, true), true);
125-
} else {
126-
// TODO support more elaborate union types
127-
return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), true);
128126
}
127+
return typeFactory.createTypeWithNullability(typeFactory.createStructType(schema.getTypes().stream()
128+
.filter(x -> x.getType() != Schema.Type.NULL)
129+
.map(x -> new AbstractMap.SimpleEntry<>(x.getName(), rel(x, typeFactory, isNullable)))
130+
.filter(x -> x.getValue().getSqlTypeName() != SqlTypeName.NULL)
131+
.filter(x -> x.getValue().getSqlTypeName() != unknown.getSqlTypeName())
132+
.collect(Collectors.toList())), isNullable);
129133
default:
130-
return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), nullable);
134+
return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), true);
131135
}
132136
}
133137

hoptimator-catalog/src/test/java/com/linkedin/hoptimator/catalog/AvroConverterTest.java

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,4 +45,23 @@ public void convertsNestedSchemas() {
4545
RelDataType rel4 = AvroConverter.rel(avroSchema4);
4646
assertTrue("types match", RelOptUtil.eq("rel4", rel4, "rel1", rel1, Litmus.THROW));
4747
}
48+
49+
@Test
50+
public void convertsNestedUnionSchemas() {
51+
String schemaString = "{\"type\":\"record\",\"name\":\"record\",\"namespace\":\"ns\",\"fields\":[{\"name\":\"event\",\"type\":[{\"type\":\"record\",\"name\":\"record_event1\",\"fields\":[{\"name\":\"strField\",\"type\":\"string\"}]},{\"type\":\"record\",\"name\":\"record_event2\",\"fields\":[{\"name\":\"strField\",\"type\":\"string\"}]}]}]}";
52+
Schema avroSchema1 = (new Schema.Parser()).parse(schemaString);
53+
RelDataType rel1 = AvroConverter.rel(avroSchema1);
54+
assertEquals(rel1.toString(), rel1.getFieldCount(), avroSchema1.getFields().size());
55+
assertNotNull(rel1.toString(), rel1.getField("event", false, false));
56+
RelDataType rel2 = rel1.getField("event", false, false).getType();
57+
assertTrue(rel2.isStruct());
58+
Schema avroSchema2 = avroSchema1.getField("event").schema();
59+
assertEquals(rel2.toString(), rel2.getFieldCount(), avroSchema2.getTypes().size());
60+
RelDataType rel3 = rel2.getField("record_event1", false, false).getType();
61+
Schema avroSchema3 = avroSchema2.getTypes().get(0);
62+
assertEquals(rel3.toString(), rel3.getFieldCount(), avroSchema3.getFields().size());
63+
Schema avroSchema4 = AvroConverter.avro("NS", "R", rel1);
64+
assertFalse("!avroSchema4.isNullable()", avroSchema4.isNullable());
65+
assertEquals(avroSchema4.toString(), avroSchema4.getFields().size(), rel1.getFieldCount());
66+
}
4867
}

hoptimator-util/src/main/java/com/linkedin/hoptimator/util/DataTypeUtils.java

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,9 @@
1515

1616
public final class DataTypeUtils {
1717

18+
private static final String MAP_KEY_TYPE = "keyType";
19+
private static final String MAP_VALUE_TYPE = "valueType";
20+
1821
private DataTypeUtils() {
1922
}
2023

@@ -46,13 +49,17 @@ private static void flattenInto(RelDataTypeFactory typeFactory, RelDataType data
4649
flattenInto(typeFactory, field.getType(), builder, Stream.concat(path.stream(),
4750
Stream.of(field.getName())).collect(Collectors.toList()));
4851
}
49-
} else if (!dataType.isStruct()) {
50-
builder.add(String.join("$", path), dataType);
51-
} else {
52+
} else if (dataType.isStruct()) {
5253
for (RelDataTypeField field : dataType.getFieldList()) {
5354
flattenInto(typeFactory, field.getType(), builder, Stream.concat(path.stream(),
5455
Stream.of(field.getName())).collect(Collectors.toList()));
5556
}
57+
} else if (dataType.getKeyType() != null && dataType.getValueType() != null) {
58+
builder.add(String.join("$", path) + "$" + MAP_KEY_TYPE, dataType.getKeyType());
59+
flattenInto(typeFactory, dataType.getValueType(), builder, Stream.concat(path.stream(),
60+
Stream.of(MAP_VALUE_TYPE)).collect(Collectors.toList()));
61+
} else {
62+
builder.add(String.join("$", path), dataType);
5663
}
5764
}
5865

@@ -86,6 +93,13 @@ private static RelDataType buildRecord(Node node, RelDataTypeFactory typeFactory
8693
return node.dataType;
8794
}
8895
RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
96+
// Placeholders to handle MAP type
97+
if (node.children.size() == 2
98+
&& node.children.containsKey(MAP_KEY_TYPE) && node.children.containsKey(MAP_VALUE_TYPE)) {
99+
RelDataType keyType = buildRecord(node.children.get(MAP_KEY_TYPE), typeFactory);
100+
RelDataType valueType = buildRecord(node.children.get(MAP_VALUE_TYPE), typeFactory);
101+
return typeFactory.createMapType(keyType, valueType);
102+
}
89103
for (Map.Entry<String, Node> child : node.children.entrySet()) {
90104
builder.add(child.getKey(), buildRecord(child.getValue(), typeFactory));
91105
}

hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -14,10 +14,9 @@
1414
import org.apache.calcite.rel.RelNode;
1515
import org.apache.calcite.rel.type.RelDataType;
1616
import org.apache.calcite.rel.type.RelDataTypeField;
17-
import org.apache.calcite.util.Pair;
17+
import org.apache.calcite.runtime.ImmutablePairList;
1818

1919
import com.google.common.annotations.VisibleForTesting;
20-
import com.google.common.collect.ImmutableList;
2120

2221
import com.linkedin.hoptimator.Deployable;
2322
import com.linkedin.hoptimator.Job;
@@ -49,15 +48,15 @@ public interface PipelineRel extends RelNode {
4948

5049
/** Implements a deployable Pipeline. */
5150
class Implementor {
52-
private final ImmutableList<Pair<Integer, String>> targetFields;
51+
private final ImmutablePairList<Integer, String> targetFields;
5352
private final Map<Source, RelDataType> sources = new LinkedHashMap<>();
5453
private RelNode query;
5554
private String sinkDatabase = "pipeline";
5655
private List<String> sinkPath = Arrays.asList(new String[]{"PIPELINE", "SINK"});
5756
private RelDataType sinkRowType = null;
5857
private Map<String, String> sinkOptions = Collections.emptyMap();
5958

60-
public Implementor(ImmutableList<Pair<Integer, String>> targetFields) {
59+
public Implementor(ImmutablePairList<Integer, String> targetFields) {
6160
this.targetFields = targetFields;
6261
}
6362

hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@
99
import java.util.Set;
1010
import java.util.function.Function;
1111
import java.util.stream.Collectors;
12-
import java.util.stream.Stream;
1312

1413
import org.apache.calcite.rel.RelNode;
1514
import org.apache.calcite.rel.core.RelFactories;
@@ -20,13 +19,15 @@
2019
import org.apache.calcite.rel.type.RelDataTypeField;
2120
import org.apache.calcite.rel.type.RelDataTypeSystem;
2221
import org.apache.calcite.rex.RexNode;
22+
import org.apache.calcite.runtime.ImmutablePairList;
2323
import org.apache.calcite.sql.SqlBasicTypeNameSpec;
2424
import org.apache.calcite.sql.SqlCall;
2525
import org.apache.calcite.sql.SqlCollectionTypeNameSpec;
2626
import org.apache.calcite.sql.SqlDataTypeSpec;
2727
import org.apache.calcite.sql.SqlDialect;
2828
import org.apache.calcite.sql.SqlIdentifier;
2929
import org.apache.calcite.sql.SqlKind;
30+
import org.apache.calcite.sql.SqlMapTypeNameSpec;
3031
import org.apache.calcite.sql.SqlNode;
3132
import org.apache.calcite.sql.SqlNodeList;
3233
import org.apache.calcite.sql.SqlRowTypeNameSpec;
@@ -42,11 +43,8 @@
4243
import org.apache.calcite.sql.type.SqlTypeName;
4344
import org.apache.calcite.sql.util.SqlShuttle;
4445
import org.apache.calcite.tools.RelBuilder;
45-
import org.apache.calcite.util.Pair;
4646
import org.apache.calcite.util.Util;
4747

48-
import com.google.common.collect.ImmutableList;
49-
5048

5149
/**
5250
* An abstract way to write SQL scripts.
@@ -103,7 +101,7 @@ default ScriptImplementor database(String database) {
103101
}
104102

105103
/** Append an insert statement, e.g. `INSERT INTO ... SELECT ...` */
106-
default ScriptImplementor insert(String schema, String table, RelNode relNode, ImmutableList<Pair<Integer, String>> targetFields) {
104+
default ScriptImplementor insert(String schema, String table, RelNode relNode, ImmutablePairList<Integer, String> targetFields) {
107105
return with(new InsertImplementor(schema, table, relNode, targetFields));
108106
}
109107

@@ -262,9 +260,9 @@ class InsertImplementor implements ScriptImplementor {
262260
private final String schema;
263261
private final String table;
264262
private final RelNode relNode;
265-
private final ImmutableList<Pair<Integer, String>> targetFields;
263+
private final ImmutablePairList<Integer, String> targetFields;
266264

267-
public InsertImplementor(String schema, String table, RelNode relNode, ImmutableList<Pair<Integer, String>> targetFields) {
265+
public InsertImplementor(String schema, String table, RelNode relNode, ImmutablePairList<Integer, String> targetFields) {
268266
this.schema = schema;
269267
this.table = table;
270268
this.relNode = relNode;
@@ -283,10 +281,10 @@ public void implement(SqlWriter w) {
283281

284282
// Drops NULL fields
285283
// Drops non-target columns, for use case: INSERT INTO (col1, col2) SELECT * FROM ...
286-
private static RelNode dropFields(RelNode relNode, ImmutableList<Pair<Integer, String>> targetFields) {
284+
private static RelNode dropFields(RelNode relNode, ImmutablePairList<Integer, String> targetFields) {
287285
List<Integer> cols = new ArrayList<>();
288286
int i = 0;
289-
Set<String> targetFieldNames = targetFields.stream().map(x -> x.right).collect(Collectors.toSet());
287+
Set<String> targetFieldNames = targetFields.stream().map(Map.Entry::getValue).collect(Collectors.toSet());
290288
for (RelDataTypeField field : relNode.getRowType().getFieldList()) {
291289
if (targetFieldNames.contains(field.getName())
292290
&& !field.getType().getSqlTypeName().equals(SqlTypeName.NULL)) {
@@ -463,6 +461,9 @@ private static SqlDataTypeSpec toSpec(RelDataType dataType) {
463461
.map(RelDataType::getSqlTypeName)
464462
.orElseThrow(() -> new IllegalArgumentException("not a collection?")), SqlParserPos.ZERO),
465463
dataType.getSqlTypeName(), SqlParserPos.ZERO), SqlParserPos.ZERO));
464+
} else if (dataType.getKeyType() != null && dataType.getValueType() != null) {
465+
return maybeNullable(dataType, new SqlDataTypeSpec(new SqlMapTypeNameSpec(
466+
toSpec(dataType.getKeyType()), toSpec(dataType.getValueType()), SqlParserPos.ZERO), SqlParserPos.ZERO));
466467
} else {
467468
return maybeNullable(dataType,
468469
new SqlDataTypeSpec(new SqlBasicTypeNameSpec(dataType.getSqlTypeName(), SqlParserPos.ZERO),

hoptimator-util/src/test/java/com/linkedin/hoptimator/util/TestDataTypeUtils.java

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -97,4 +97,46 @@ public void flattenUnflattenNestedArrays() {
9797
+ "`CAR` FLOAT ARRAY) WITH ();", unflattenedConnector,
9898
"Flattened-unflattened connector should be correct");
9999
}
100+
101+
@Test
102+
public void flattenUnflattenComplexMap() {
103+
RelDataTypeFactory typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
104+
RelDataTypeFactory.Builder mapValue = new RelDataTypeFactory.Builder(typeFactory);
105+
mapValue.add("BAR", SqlTypeName.VARCHAR);
106+
mapValue.add("CAR", SqlTypeName.INTEGER);
107+
108+
RelDataTypeFactory.Builder keyBuilder = new RelDataTypeFactory.Builder(typeFactory);
109+
RelDataTypeFactory.Builder valueBuilder = new RelDataTypeFactory.Builder(typeFactory);
110+
keyBuilder.add("QUX", SqlTypeName.VARCHAR);
111+
valueBuilder.add("QIZ", mapValue.build());
112+
113+
RelDataTypeFactory.Builder mapBuilder = new RelDataTypeFactory.Builder(typeFactory);
114+
mapBuilder.add("FOO", typeFactory.createMapType(keyBuilder.build(), valueBuilder.build()));
115+
RelDataType rowType = mapBuilder.build();
116+
Assertions.assertEquals(1, rowType.getFieldList().size());
117+
RelDataType flattenedType = DataTypeUtils.flatten(rowType, typeFactory);
118+
Assertions.assertEquals(3, flattenedType.getFieldList().size());
119+
List<String> flattenedNames = flattenedType.getFieldList().stream().map(RelDataTypeField::getName)
120+
.collect(Collectors.toList());
121+
Assertions.assertIterableEquals(Arrays.asList("FOO$keyType", "FOO$valueType$QIZ$BAR", "FOO$valueType$QIZ$CAR"), flattenedNames);
122+
String flattenedConnector = new ScriptImplementor.ConnectorImplementor("S", "T1",
123+
flattenedType, Collections.emptyMap()).sql();
124+
Assertions.assertEquals("CREATE TABLE IF NOT EXISTS `S`.`T1` ("
125+
+ "`FOO_keyType` ROW(`QUX` VARCHAR), "
126+
+ "`FOO_valueType_QIZ_BAR` VARCHAR, "
127+
+ "`FOO_valueType_QIZ_CAR` INTEGER) WITH ();", flattenedConnector,
128+
"Flattened connector should have simplified map");
129+
130+
RelDataType unflattenedType = DataTypeUtils.unflatten(flattenedType, typeFactory);
131+
RelOptUtil.eq("original", rowType, "flattened-unflattened", unflattenedType, Litmus.THROW);
132+
String originalConnector = new ScriptImplementor.ConnectorImplementor("S", "T1",
133+
rowType, Collections.emptyMap()).sql();
134+
String unflattenedConnector = new ScriptImplementor.ConnectorImplementor("S", "T1",
135+
unflattenedType, Collections.emptyMap()).sql();
136+
Assertions.assertEquals(originalConnector, unflattenedConnector,
137+
"Flattening and unflattening data types should have no impact on connector");
138+
Assertions.assertEquals("CREATE TABLE IF NOT EXISTS `S`.`T1` ("
139+
+ "`FOO` MAP< ROW(`QUX` VARCHAR), ROW(`QIZ` ROW(`BAR` VARCHAR, `CAR` INTEGER)) >) WITH ();", unflattenedConnector,
140+
"Flattened-unflattened connector should be correct");
141+
}
100142
}

0 commit comments

Comments
 (0)