-
Notifications
You must be signed in to change notification settings - Fork 3k
Add flink data type convertor to convert data types between iceberg data type and flink data type. #1096
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
…ata type and flink data type.
build.gradle
Outdated
| compile project(':iceberg-orc') | ||
| compile project(':iceberg-parquet') | ||
| compile project(':iceberg-arrow') | ||
| compile "org.apache.flink:flink-streaming-java_2.11::tests" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does Flink have a Scala 2.12 version? Will it cause a problem to rely on 2.11?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes we have scala 2.12 version. I'm not sure what's the specific problem you mean in 2.11, You have more information ? thanks.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
IMO, we may need to load the corresponding flink Jars (which is depending on scala ) for the configured scala version. For example, if we choose to use the scala 2.12, then the iceberg-flink module should choose to use all the flink-xxx_2.12 jars (should throw error if the flink-xx_2.12 does not exist).
This may need to write few gradlew scripts or methods. I think it could be considered in a separate issue. There're also some other modules such as iceberg-spark which should handle this issue too.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My main concern is that if we compile against 2.11, it won't work in 2.12. Then we would need two separate builds.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think so, building two iceberg-flink jars and deploying them to the repo, so that the downstream users can choose to use the correct iceberg-flink jar based on their scala version.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I didn't intend to ask you to add another module to the build. Sorry about not being clear.
In many cases, it's possible to work with both 2.11 and 2.12 with compiled binaries. We have two internal versions of Spark that we do this with, although Spark defines the catalog and table APIs in Java so we have less to worry about. Ideally, we would have one module that works with both and we would have some plan to test that. Otherwise, it's probably a good idea to move this code into Flink as soon as possible because it makes more sense to maintain just source compatibility there.
For now, I think we should consider options for testing this against both 2.11 and 2.12. Maybe we can add a build property to switch between 2.11 and 2.12 and just test both in CI. Until then, I think we should just build for 2.12. 2.11 hasn't been supported since 2017, so it makes sense for new development to target 2.12.
flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
| } | ||
| } | ||
|
|
||
| public T fields(FieldsDataType dataType, Map<String, Tuple2<String, T>> fieldResults) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If fields is always a row, it seems to me that row might be a better name because it is more clear that it is a record/struct/tuple type. It wasn't clear to me that fields was the record type, since most other formats make a distinction between a record and its field list.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The name fields is matching the FieldsDataType , like the collection is matching the CollectionDataType, map is matching the KeyValueDataType.
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
Outdated
Show resolved
Hide resolved
| Types.NestedField.optional(17, "decimal", Types.DecimalType.of(2, 2)) | ||
| ); | ||
|
|
||
| Assert.assertEquals(expectedSchema.toString(), actualSchema.toString()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Instead of using toString, use asStruct. Struct implements equals so you can use it in tests.
flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java
Outdated
Show resolved
Hide resolved
build.gradle
Outdated
| compile project(':iceberg-orc') | ||
| compile project(':iceberg-parquet') | ||
| compile project(':iceberg-arrow') | ||
| compile "org.apache.flink:flink-streaming-java_2.11::tests" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes we have scala 2.12 version. I'm not sure what's the specific problem you mean in 2.11, You have more information ? thanks.
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
Outdated
Show resolved
Hide resolved
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
| } | ||
| } | ||
|
|
||
| public T fields(FieldsDataType dataType, Map<String, Tuple2<String, T>> fieldResults) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The name fields is matching the FieldsDataType , like the collection is matching the CollectionDataType, map is matching the KeyValueDataType.
flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java
Outdated
Show resolved
Hide resolved
versions.props
Outdated
| @@ -1,5 +1,6 @@ | |||
| org.slf4j:* = 1.7.25 | |||
| org.apache.avro:avro = 1.9.2 | |||
| org.apache.flink:* = 1.10.0 | |||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The flink version can be upgraded to 1.10.1 now.
flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
Outdated
Show resolved
Hide resolved
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(struct.fields(), aliasToId == null ? 0 : aliasToId.hashCode()); | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Got it, thanks.
| FieldsDataType root = (FieldsDataType) schema.toRowDataType(); | ||
| Type converted = FlinkTypeVisitor.visit(root, new FlinkTypeToType(root)); | ||
|
|
||
| return new Schema(converted.asNestedType().asStructType().fields()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
asNestedType is no longer necessary. Now asStructType is defined on Type, not just NestedType.
|
|
||
| int index = 0; | ||
| for (int i = 0; i < rowFields.size(); i++) { | ||
| int id = isRoot ? index : getNextId(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why have variable index when its value is always equal to i?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK, the index can be removed now (The first version I used a for (String name : types.keySet()) so defined a index variable) .
build.gradle
Outdated
| } | ||
| } | ||
|
|
||
| project(':iceberg-flink_2.12') { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is no source code in flink_2.12, so this module is actually empty. Like I said in the other comment, let's have one 2.12 module for now to get this PR in. Then we can figure out how to test compatibility with 2.11 later.
…taTypes.ARRAY (it should depend on the nullbility of elementType.)
|
This looks ready to go once we deduplicate the Flink modules. Whether we depend on 2.11 or 2.12 doesn't matter to me, but I think we should just have one for now, like I said in the thread. |
|
Hi @rdblue , I've removed the flink module with scala 2.11 as you suggested. I've one minor question, when I generate the dependencies.lock file, seems it contains all the That makes the |
|
Ping @rdblue , any other concern about this pull request ? If no more concern, please help to merge this patch so that I could PR the following patches, Thanks in advance. |
|
Ping @rdblue. |
|
Looks good now. Thanks, @openinx! Sorry for the delay getting back to this for a review. I was reviewing a few other big patches in the mean time. |
JingsongLi
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
|
|
||
| public class FlinkTypeVisitor<T> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Flink has LogicalTypeVisitor and DataTypeVisitor, they are very useful for visiting types and you don't need have this static <T> T visit method, it is not so elegant.
And for FieldsDataType, it not has a good design in 1.9 and 1.10, so in Flink 1.11, it has been refactored to be removed getFieldDataTypes.
And I think maybe a LogicalTypeVisitor is enough, since we never touch the physical information in the DataTypes.
| this.nextId = root.getFieldDataTypes().size(); | ||
| } | ||
|
|
||
| private int getNextId() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A better name is generateNextId or just nextId. Looks like it is not only "get".
| } | ||
|
|
||
| @Override | ||
| public Type collection(CollectionDataType collection, Type elementType) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A CollectionDataType may be MultisetType too. Maybe iceberg not support it now? Or we can map it to Map<T, Integer>.
|
|
||
| @Override | ||
| public Type map(KeyValueDataType map, Type keyType, Type valueType) { | ||
| // keys in map are not allowed to be null. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But flink can be, so should we throw unsupported exception here?
@waterlx have did the netflix Poc before. Had a discussion with him before, we're planning to divide the flink connector module into serveral patches, and push it into the offical apache repo.
This is the first pull requset, which would align the data types between flink & iceberg (also introduced the iceberg-flink module with necessary flink dependencies). The following pull requests would be:
For the part-2, we may better to have a doc describe the design behind it because it seems need some flink background to understand the flink operator state and it's design.