Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit e51f28f

Browse files
mxmcogwirrel
authored andcommitted
Flink: Backport Dynamic Iceberg Sink: Add table update code for schema comparison and evolution to Flink 1.19 / 1.20 (apache#13247)
Backports apache#13032
1 parent bfaa45e commit e51f28f

20 files changed

+4698
-0
lines changed
Lines changed: 266 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,266 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.iceberg.flink.sink.dynamic;
20+
21+
import java.util.List;
22+
import java.util.Map;
23+
import org.apache.iceberg.Schema;
24+
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
25+
import org.apache.iceberg.schema.SchemaWithPartnerVisitor;
26+
import org.apache.iceberg.types.Type;
27+
import org.apache.iceberg.types.Types;
28+
29+
/**
30+
* Visitor class which compares an input schema to a table schema and emits a compatibility {@link
31+
* Result}.
32+
*
33+
* <ul>
34+
* <li>SAME: The two schemas are semantically identical
35+
* <li>DATA_CONVERSION_NEEDED: We can evolve the data associated with the input schema to match
36+
* the table schema.
37+
* <li>SCHEMA_UPDATE_NEEDED: We need to migrate the table schema to match the input schema.
38+
* </ul>
39+
*
40+
* The input schema fields are compared to the table schema via their names.
41+
*/
42+
public class CompareSchemasVisitor
43+
extends SchemaWithPartnerVisitor<Integer, CompareSchemasVisitor.Result> {
44+
45+
private final Schema tableSchema;
46+
47+
private CompareSchemasVisitor(Schema tableSchema) {
48+
this.tableSchema = tableSchema;
49+
}
50+
51+
public static Result visit(Schema dataSchema, Schema tableSchema) {
52+
return visit(dataSchema, tableSchema, true);
53+
}
54+
55+
public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) {
56+
return visit(
57+
dataSchema,
58+
-1,
59+
new CompareSchemasVisitor(tableSchema),
60+
new PartnerIdByNameAccessors(tableSchema, caseSensitive));
61+
}
62+
63+
@Override
64+
public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream) {
65+
if (tableSchemaId == null) {
66+
return Result.SCHEMA_UPDATE_NEEDED;
67+
}
68+
69+
return downstream;
70+
}
71+
72+
@Override
73+
public Result struct(Types.StructType struct, Integer tableSchemaId, List<Result> fields) {
74+
if (tableSchemaId == null) {
75+
return Result.SCHEMA_UPDATE_NEEDED;
76+
}
77+
78+
Result result = fields.stream().reduce(Result::merge).orElse(Result.SCHEMA_UPDATE_NEEDED);
79+
80+
if (result == Result.SCHEMA_UPDATE_NEEDED) {
81+
return Result.SCHEMA_UPDATE_NEEDED;
82+
}
83+
84+
Type tableSchemaType =
85+
tableSchemaId == -1 ? tableSchema.asStruct() : tableSchema.findField(tableSchemaId).type();
86+
if (!tableSchemaType.isStructType()) {
87+
return Result.SCHEMA_UPDATE_NEEDED;
88+
}
89+
90+
if (struct.fields().size() != tableSchemaType.asStructType().fields().size()) {
91+
return Result.DATA_CONVERSION_NEEDED;
92+
}
93+
94+
for (int i = 0; i < struct.fields().size(); ++i) {
95+
if (!struct
96+
.fields()
97+
.get(i)
98+
.name()
99+
.equals(tableSchemaType.asStructType().fields().get(i).name())) {
100+
return Result.DATA_CONVERSION_NEEDED;
101+
}
102+
}
103+
104+
return result;
105+
}
106+
107+
@Override
108+
public Result field(Types.NestedField field, Integer tableSchemaId, Result typeResult) {
109+
if (tableSchemaId == null) {
110+
return Result.SCHEMA_UPDATE_NEEDED;
111+
}
112+
113+
if (typeResult != Result.SAME) {
114+
return typeResult;
115+
}
116+
117+
if (tableSchema.findField(tableSchemaId).isRequired() && field.isOptional()) {
118+
return Result.SCHEMA_UPDATE_NEEDED;
119+
} else {
120+
return Result.SAME;
121+
}
122+
}
123+
124+
@Override
125+
public Result list(Types.ListType list, Integer tableSchemaId, Result elementsResult) {
126+
if (tableSchemaId == null) {
127+
return Result.SCHEMA_UPDATE_NEEDED;
128+
}
129+
130+
return elementsResult;
131+
}
132+
133+
@Override
134+
public Result map(
135+
Types.MapType map, Integer tableSchemaId, Result keyResult, Result valueResult) {
136+
if (tableSchemaId == null) {
137+
return Result.SCHEMA_UPDATE_NEEDED;
138+
}
139+
140+
return keyResult.merge(valueResult);
141+
}
142+
143+
@Override
144+
@SuppressWarnings("checkstyle:CyclomaticComplexity")
145+
public Result primitive(Type.PrimitiveType primitive, Integer tableSchemaId) {
146+
if (tableSchemaId == null) {
147+
return Result.SCHEMA_UPDATE_NEEDED;
148+
}
149+
150+
Type tableSchemaType = tableSchema.findField(tableSchemaId).type();
151+
if (!tableSchemaType.isPrimitiveType()) {
152+
return Result.SCHEMA_UPDATE_NEEDED;
153+
}
154+
155+
Type.PrimitiveType tableSchemaPrimitiveType = tableSchemaType.asPrimitiveType();
156+
if (primitive.equals(tableSchemaPrimitiveType)) {
157+
return Result.SAME;
158+
} else if (primitive.equals(Types.IntegerType.get())
159+
&& tableSchemaPrimitiveType.equals(Types.LongType.get())) {
160+
return Result.DATA_CONVERSION_NEEDED;
161+
} else if (primitive.equals(Types.FloatType.get())
162+
&& tableSchemaPrimitiveType.equals(Types.DoubleType.get())) {
163+
return Result.DATA_CONVERSION_NEEDED;
164+
} else if (primitive.equals(Types.DateType.get())
165+
&& tableSchemaPrimitiveType.equals(Types.TimestampType.withoutZone())) {
166+
return Result.DATA_CONVERSION_NEEDED;
167+
} else if (primitive.typeId() == Type.TypeID.DECIMAL
168+
&& tableSchemaPrimitiveType.typeId() == Type.TypeID.DECIMAL) {
169+
Types.DecimalType dataType = (Types.DecimalType) primitive;
170+
Types.DecimalType tableType = (Types.DecimalType) tableSchemaPrimitiveType;
171+
return dataType.scale() == tableType.scale() && dataType.precision() < tableType.precision()
172+
? Result.DATA_CONVERSION_NEEDED
173+
: Result.SCHEMA_UPDATE_NEEDED;
174+
} else {
175+
return Result.SCHEMA_UPDATE_NEEDED;
176+
}
177+
}
178+
179+
static class PartnerIdByNameAccessors implements PartnerAccessors<Integer> {
180+
private final Schema tableSchema;
181+
private boolean caseSensitive = true;
182+
183+
PartnerIdByNameAccessors(Schema tableSchema) {
184+
this.tableSchema = tableSchema;
185+
}
186+
187+
private PartnerIdByNameAccessors(Schema tableSchema, boolean caseSensitive) {
188+
this(tableSchema);
189+
this.caseSensitive = caseSensitive;
190+
}
191+
192+
@Override
193+
public Integer fieldPartner(Integer tableSchemaFieldId, int fieldId, String name) {
194+
Types.StructType struct;
195+
if (tableSchemaFieldId == -1) {
196+
struct = tableSchema.asStruct();
197+
} else {
198+
struct = tableSchema.findField(tableSchemaFieldId).type().asStructType();
199+
}
200+
201+
Types.NestedField field =
202+
caseSensitive ? struct.field(name) : struct.caseInsensitiveField(name);
203+
if (field != null) {
204+
return field.fieldId();
205+
}
206+
207+
return null;
208+
}
209+
210+
@Override
211+
public Integer mapKeyPartner(Integer tableSchemaMapId) {
212+
Types.NestedField mapField = tableSchema.findField(tableSchemaMapId);
213+
if (mapField != null) {
214+
return mapField.type().asMapType().fields().get(0).fieldId();
215+
}
216+
217+
return null;
218+
}
219+
220+
@Override
221+
public Integer mapValuePartner(Integer tableSchemaMapId) {
222+
Types.NestedField mapField = tableSchema.findField(tableSchemaMapId);
223+
if (mapField != null) {
224+
return mapField.type().asMapType().fields().get(1).fieldId();
225+
}
226+
227+
return null;
228+
}
229+
230+
@Override
231+
public Integer listElementPartner(Integer tableSchemaListId) {
232+
Types.NestedField listField = tableSchema.findField(tableSchemaListId);
233+
if (listField != null) {
234+
return listField.type().asListType().fields().get(0).fieldId();
235+
}
236+
237+
return null;
238+
}
239+
}
240+
241+
public enum Result {
242+
SAME(0),
243+
DATA_CONVERSION_NEEDED(1),
244+
SCHEMA_UPDATE_NEEDED(2);
245+
246+
private static final Map<Integer, Result> BY_ID = Maps.newHashMap();
247+
248+
static {
249+
for (Result e : Result.values()) {
250+
if (BY_ID.put(e.id, e) != null) {
251+
throw new IllegalArgumentException("Duplicate id: " + e.id);
252+
}
253+
}
254+
}
255+
256+
private final int id;
257+
258+
Result(int id) {
259+
this.id = id;
260+
}
261+
262+
private Result merge(Result other) {
263+
return BY_ID.get(Math.max(this.id, other.id));
264+
}
265+
}
266+
}

0 commit comments

Comments
 (0)