rdblue commented on a change in pull request #1096:
URL: https://github.com/apache/iceberg/pull/1096#discussion_r436852808
##########
File path: build.gradle
##########
@@ -235,6 +235,38 @@ project(':iceberg-data') {
}
}
+project(':iceberg-flink') {
+ apply plugin: 'scala'
+
+ dependencies {
+ compile project(':iceberg-api')
+ compile project(':iceberg-common')
+ compile project(':iceberg-core')
+ compile project(':iceberg-data')
+ compile project(':iceberg-orc')
+ compile project(':iceberg-parquet')
+ compile project(':iceberg-arrow')
+ compile "org.apache.flink:flink-streaming-java_2.11::tests"
Review comment:
This is compiling the Iceberg sink against the Flink Scala 2.11 API,
right? But Scala isn't binary compatible between minor releases, so Jars
compiled for 2.11 may not work with 2.12. How should we avoid problems with
binary compatibility?
##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.types.AtomicDataType;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.KeyValueDataType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+
+public class FlinkTypeToType extends FlinkTypeVisitor<Type> {
+ private final FieldsDataType root;
+ private int nextId = 0;
+
+ FlinkTypeToType(FieldsDataType root) {
+ this.root = root;
+ // the root struct's fields use the first ids
+ this.nextId = root.getFieldDataTypes().size();
+ }
+
+ private int getNextId() {
+ int next = nextId;
+ nextId += 1;
+ return next;
+ }
+
+ @Override
+ public Type fields(FieldsDataType dataType, Map<String, Tuple2<String,
Type>> types) {
+ List<Types.NestedField> newFields =
Lists.newArrayListWithExpectedSize(types.size());
+ boolean isRoot = root == dataType;
+
+ Map<String, DataType> fieldsMap = dataType.getFieldDataTypes();
+ int index = 0;
+ for (String name : types.keySet()) {
+ assert fieldsMap.containsKey(name);
+ DataType field = fieldsMap.get(name);
+ Tuple2<String, Type> tuple2 = types.get(name);
+
+ int id = isRoot ? index : getNextId();
+ if (field.getLogicalType().isNullable()) {
+ newFields.add(Types.NestedField.optional(id, name, tuple2.f1,
tuple2.f0));
+ } else {
+ newFields.add(Types.NestedField.required(id, name, tuple2.f1,
tuple2.f0));
+ }
+ index++;
+ }
+ return Types.StructType.of(newFields);
+ }
+
+ @Override
+ public Type collection(CollectionDataType collection, Type elementType) {
+ if (collection.getElementDataType().getLogicalType().isNullable()) {
+ return Types.ListType.ofOptional(getNextId(), elementType);
+ } else {
+ return Types.ListType.ofRequired(getNextId(), elementType);
+ }
+ }
+
+ @Override
+ public Type map(KeyValueDataType map, Type keyType, Type valueType) {
+ if (map.getValueDataType().getLogicalType().isNullable()) {
+ return Types.MapType.ofOptional(getNextId(), getNextId(), keyType,
valueType);
+ } else {
+ return Types.MapType.ofRequired(getNextId(), getNextId(), keyType,
valueType);
+ }
+ }
+
+ @SuppressWarnings("checkstyle:CyclomaticComplexity")
+ @Override
+ public Type atomic(AtomicDataType type) {
+ LogicalType inner = type.getLogicalType();
+ if (inner instanceof VarCharType ||
+ inner instanceof CharType) {
+ return Types.StringType.get();
+ } else if (inner instanceof BooleanType) {
+ return Types.BooleanType.get();
+ } else if (inner instanceof IntType ||
+ inner instanceof SmallIntType ||
+ inner instanceof TinyIntType) {
+ return Types.IntegerType.get();
+ } else if (inner instanceof BigIntType) {
+ return Types.LongType.get();
+ } else if (inner instanceof VarBinaryType ||
+ inner instanceof BinaryType) {
+ return Types.BinaryType.get();
+ } else if (inner instanceof FloatType) {
+ return Types.FloatType.get();
+ } else if (inner instanceof DoubleType) {
+ return Types.DoubleType.get();
+ } else if (inner instanceof DateType) {
+ return Types.DateType.get();
+ } else if (inner instanceof TimeType) {
+ return Types.TimeType.get();
+ } else if (inner instanceof TimestampType) {
Review comment:
It looks like the Flink type that most closely matches is
[LocalZonedTimestampType](https://javadoc.io/doc/org.apache.flink/flink-table-common/latest/org/apache/flink/table/types/logical/LocalZonedTimestampType.html)
because the Iceberg type does not store the original time zone.
##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.types.AtomicDataType;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.KeyValueDataType;
+import org.apache.flink.table.types.logical.RowType;
+
+public class FlinkTypeVisitor<T> {
+
+ static <T> T visit(DataType dataType, FlinkTypeVisitor<T> visitor) {
+ if (dataType instanceof FieldsDataType) {
+ FieldsDataType fieldsType = (FieldsDataType) dataType;
+ Map<String, DataType> fields = fieldsType.getFieldDataTypes();
+ Map<String, Tuple2<String, T>> fieldResults = new LinkedHashMap<>();
+ // Make sure that we're traversing the fields in the same order as
constructing the schema's fields.
+ RowType rowType = (RowType) dataType.getLogicalType();
+ for (int i = 0; i < fields.size(); i++) {
+ String name = rowType.getFieldNames().get(i);
+ String comment =
rowType.getFields().get(i).getDescription().orElse(null);
+ fieldResults.put(name, Tuple2.of(comment, visit(fields.get(name),
visitor)));
Review comment:
Looks good now.
##########
File path: api/src/main/java/org/apache/iceberg/Schema.java
##########
@@ -314,4 +315,31 @@ public String toString() {
.map(f -> " " + f)
.collect(Collectors.toList())));
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof Schema)) {
+ return false;
+ }
+
+ Schema that = (Schema) o;
+ if (!Objects.equals(struct.fields(), that.struct.fields())) {
+ return false;
+ }
+
+ if (aliasToId == that.aliasToId) {
+ return true;
+ }
+
+ if (aliasToId == null || that.aliasToId == null) {
+ return false;
+ }
+
+ return aliasToId.equals(that.aliasToId);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(struct.fields(), aliasToId == null ? 0 :
aliasToId.hashCode());
+ }
Review comment:
`Schema` doesn't implement `equals` because it isn't clear what schema
equality means. Because we track fields by ID, two schemas are equal only if
their fields have the same IDs, but most people don't think about schema
equality that way and think of a SQL schema, like `id bigint, data string`. To
avoid confusion, we don't provide an `equals` method that may have confusing
results. Instead, we implement `equals` for structs so you can use
`schema.asStruct()` in test assertions.
##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink;
+
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+
+public class FlinkSchemaUtil {
+
+ private FlinkSchemaUtil() {
+ }
+
+ /**
+ * Convert the flink table schema to apache iceberg schema.
+ */
+ public static Schema convert(TableSchema schema) {
+ Preconditions.checkArgument(schema.toRowDataType() instanceof
FieldsDataType, "Should be FieldsDataType");
+
+ FieldsDataType root = (FieldsDataType) schema.toRowDataType();
+ Type converted = FlinkTypeVisitor.visit(root, new FlinkTypeToType(root));
+
+ return new Schema(converted.asNestedType().asStructType().fields());
Review comment:
`asNestedType` is no longer necessary. Now `asStructType` is defined on
`Type`, not just `NestedType`.
##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink;
+
+import java.util.List;
+import org.apache.flink.table.types.AtomicDataType;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.KeyValueDataType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.ZonedTimestampType;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+
+public class FlinkTypeToType extends FlinkTypeVisitor<Type> {
+ private final FieldsDataType root;
+ private int nextId = 0;
+
+ FlinkTypeToType(FieldsDataType root) {
+ this.root = root;
+ // the root struct's fields use the first ids
+ this.nextId = root.getFieldDataTypes().size();
+ }
+
+ private int getNextId() {
+ int next = nextId;
+ nextId += 1;
+ return next;
+ }
+
+ @Override
+ public Type fields(FieldsDataType fields, List<Type> types) {
+ List<Types.NestedField> newFields =
Lists.newArrayListWithExpectedSize(types.size());
+ boolean isRoot = root == fields;
+
+ List<RowType.RowField> rowFields = ((RowType)
fields.getLogicalType()).getFields();
+ Preconditions.checkArgument(rowFields.size() == types.size(), "fields list
and types list should have same size.");
+
+ int index = 0;
+ for (int i = 0; i < rowFields.size(); i++) {
+ int id = isRoot ? index : getNextId();
Review comment:
Why have variable `index` when its value is always equal to `i`?
##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkSchemaUtil {
Review comment:
I don't see test cases for lists with required elements or lists of
structs. Can you add those?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]