Skip to content

Commit d0145b3

Browse files
[Backport][Flink] Fix mapping between Delta's BinaryType and Flink's VarBinaryType (#4554)
#### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [X] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Resolves #3977 This PR fixes a bug where the Delta-Flink connector would incorrectly map Delta's `BinaryType` (which is variable length) type to Flink's `BinaryType` (which is fixed length). Instead, this PR fixes it so that Delta's `BinaryType` is mapped to Flink's `VarBinaryType(MAX_LENGTH)`. As a comparison, Iceberg does the same [here](https://github.com/apache/iceberg/blob/8e2ffb35da2d4c5059e96cb78a30fd8c54cfbedf/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java#L125). This incorrect logic caused the behaviour seen in #3977 because: - First, we had a source (datagen) table with Flink's BYTES (which is a synonym for VarBinaryType of MAX_LENGTH) - Second, we had a target (delta) table. Flink created it with schema BYTES and the Delta schema had delta type `BinaryType`. - All good so far. No problems yet. - When we tried to `INSERT INTO <target> SELECT * FROM <source>`, the DeltaCatalog would lookup the "flink schema" of the target Delta table. It would see Delta's BinaryType and map it incorrectly to flink's BinaryType (not Flink's BYTES or VarBinaryType) - Hence it would throw the error below ``` org.apache.flink.table.api.ValidationException: Column types of query result and sink for 'print_sink' do not match. Cause: Incompatible types for sink column 'binary_data' at position 1. Query schema: [id: BIGINT, binary_data: BYTES] Sink schema: [id: BIGINT, binary_data: BINARY(1)] ``` ## How was this patch tested? Updated schema conversion tests. New e2e test which directly tests the problematic scenario brought up in #3977. ## Does this PR introduce _any_ user-facing changes? Yes. We fix the mapping of Delta's BinaryType to flinks VarBinaryTYpe <!-- Thanks for sending a pull request! Here are some tips for you: 1. If this is your first time, please read our contributor guidelines: https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md 2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP] Your PR title ...'. 3. Be sure to keep the PR description updated to reflect all changes. 4. Please write your PR title to summarize what this PR proposes. 5. If possible, provide a concise example to reproduce the issue for a faster review. 6. If applicable, include the corresponding issue number in the PR title and link it in the body. --> #### Which Delta project/connector is this regarding? <!-- Please add the component selected below to the beginning of the pull request title For example: [Spark] Title of my pull request --> - [ ] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description <!-- - Describe what this PR changes. - Describe why we need the change. If this PR resolves an issue be sure to include "Resolves #XXX" to correctly link and close the issue upon merge. --> ## How was this patch tested? <!-- If tests were added, say they were added here. Please make sure to test the changes thoroughly including negative and positive cases if possible. If the changes were tested in any way other than unit tests, please clarify how you tested step by step (ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future). If the changes were not tested, please explain why. --> ## Does this PR introduce _any_ user-facing changes? <!-- If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible. If possible, please also clarify if this is a user-facing change compared to the released Delta Lake versions or within the unreleased branches such as master. If no, write 'No'. --> Co-authored-by: Scott Sandre <[email protected]>
1 parent 196fce0 commit d0145b3

File tree

3 files changed

+57
-5
lines changed

3 files changed

+57
-5
lines changed

connectors/flink/src/main/java/io/delta/flink/source/internal/SchemaConverter.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,6 @@
22

33
import org.apache.flink.table.types.logical.ArrayType;
44
import org.apache.flink.table.types.logical.BigIntType;
5-
import org.apache.flink.table.types.logical.BinaryType;
65
import org.apache.flink.table.types.logical.BooleanType;
76
import org.apache.flink.table.types.logical.DateType;
87
import org.apache.flink.table.types.logical.DecimalType;
@@ -16,6 +15,7 @@
1615
import org.apache.flink.table.types.logical.SmallIntType;
1716
import org.apache.flink.table.types.logical.TimestampType;
1817
import org.apache.flink.table.types.logical.TinyIntType;
18+
import org.apache.flink.table.types.logical.VarBinaryType;
1919
import org.apache.flink.table.types.logical.VarCharType;
2020

2121
import io.delta.standalone.types.DataType;
@@ -64,7 +64,7 @@ public static LogicalType toFlinkDataType(DataType deltaType, boolean nullable)
6464
case LONG:
6565
return new BigIntType(nullable);
6666
case BINARY:
67-
return new BinaryType(nullable, BinaryType.DEFAULT_LENGTH);
67+
return new VarBinaryType(nullable, VarBinaryType.MAX_LENGTH);
6868
case BOOLEAN:
6969
return new BooleanType(nullable);
7070
case BYTE:

connectors/flink/src/test/java/io/delta/flink/source/internal/SchemaConverterTest.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,6 @@
55

66
import org.apache.flink.table.types.logical.ArrayType;
77
import org.apache.flink.table.types.logical.BigIntType;
8-
import org.apache.flink.table.types.logical.BinaryType;
98
import org.apache.flink.table.types.logical.BooleanType;
109
import org.apache.flink.table.types.logical.DateType;
1110
import org.apache.flink.table.types.logical.DecimalType;
@@ -18,6 +17,7 @@
1817
import org.apache.flink.table.types.logical.SmallIntType;
1918
import org.apache.flink.table.types.logical.TimestampType;
2019
import org.apache.flink.table.types.logical.TinyIntType;
20+
import org.apache.flink.table.types.logical.VarBinaryType;
2121
import org.apache.flink.table.types.logical.VarCharType;
2222
import org.junit.jupiter.params.ParameterizedTest;
2323
import org.junit.jupiter.params.provider.Arguments;
@@ -71,7 +71,9 @@ private static Stream<Arguments> dataTypes() {
7171
Arguments.of(new io.delta.standalone.types.ByteType(), new TinyIntType()),
7272
Arguments.of(new io.delta.standalone.types.ShortType(), new SmallIntType()),
7373
Arguments.of(new io.delta.standalone.types.LongType(), new BigIntType()),
74-
Arguments.of(new io.delta.standalone.types.BinaryType(), new BinaryType()),
74+
Arguments.of(
75+
new io.delta.standalone.types.BinaryType(),
76+
new VarBinaryType(VarBinaryType.MAX_LENGTH)),
7577
Arguments.of(new io.delta.standalone.types.TimestampType(), new TimestampType()),
7678
Arguments.of(new io.delta.standalone.types.DateType(), new DateType()),
7779
Arguments.of(
@@ -163,7 +165,7 @@ private static Stream<Arguments> mapTypes() {
163165
new io.delta.standalone.types.ShortType(),
164166
true
165167
),
166-
new MapType(new BinaryType(), new SmallIntType())),
168+
new MapType(new VarBinaryType(VarBinaryType.MAX_LENGTH), new SmallIntType())),
167169
Arguments.of(
168170
new io.delta.standalone.types.MapType(
169171
new io.delta.standalone.types.StringType(),

connectors/flink/src/test/java/io/delta/flink/table/it/suite/DeltaEndToEndTableTestSuite.java

Lines changed: 50 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -94,6 +94,56 @@ public void setUp() {
9494
nonPartitionedLargeTablePath);
9595
}
9696

97+
/** Tests fix for https://github.com/delta-io/delta/issues/3977 */
98+
@Test
99+
public void testWriteFromDatagenTableToDeltaTypeWithBytesType() throws Exception {
100+
final StreamTableEnvironment tableEnv = setupTableEnvAndDeltaCatalog(false);
101+
final String targetTablePath = TEMPORARY_FOLDER.newFolder().getAbsolutePath();
102+
final String datagenSourceDDL = ""
103+
+ "CREATE TABLE source_table ("
104+
+ " id BIGINT,"
105+
+ " binary_data BYTES"
106+
+ ") WITH ("
107+
+ " 'connector' = 'datagen',"
108+
+ " 'fields.id.kind' = 'sequence',"
109+
+ " 'fields.id.start' = '1',"
110+
+ " 'fields.id.end' = '8',"
111+
+ " 'number-of-rows' = '8'," // this makes the source BOUNDED
112+
+ " 'fields.binary_data.kind' = 'random',"
113+
+ " 'fields.binary_data.length' = '16'"
114+
+ ")";
115+
final String deltaSinkDDL = String.format(""
116+
+ "CREATE TABLE target_table ("
117+
+ " id BIGINT,"
118+
+ " binary_data BYTES"
119+
+ ") WITH ("
120+
+ " 'connector' = 'delta',"
121+
+ " 'table-path' = '%s'"
122+
+ ")",
123+
targetTablePath);
124+
125+
// Stage 1: Create the source and validate it
126+
127+
tableEnv.executeSql(datagenSourceDDL).await();
128+
129+
final List<Row> sourceRows =
130+
DeltaTestUtils.readTableResult(tableEnv.executeSql("SELECT * FROM source_table"));
131+
132+
assertThat(sourceRows).hasSize(8);
133+
134+
// Stage 2: Create the sink and insert into it and validate it
135+
136+
tableEnv.executeSql(deltaSinkDDL).await();
137+
138+
// If our fix for issue #3977 did not work, then this would have thrown an exception.
139+
tableEnv.executeSql("INSERT INTO target_table SELECT * FROM source_table").await();
140+
141+
final List<Row> targetRows =
142+
DeltaTestUtils.readTableResult(tableEnv.executeSql("SELECT * FROM target_table"));
143+
144+
assertThat(targetRows).hasSize(8);
145+
}
146+
97147
@Test
98148
public void shouldReadAndWriteDeltaTable() throws Exception {
99149

0 commit comments

Comments
 (0)