18
18
*/
19
19
package org .apache .iceberg .flink ;
20
20
21
+ import java .util .Collections ;
21
22
import java .util .List ;
23
+ import java .util .Map ;
22
24
import java .util .Set ;
25
+ import java .util .UUID ;
26
+ import java .util .function .Function ;
27
+ import java .util .stream .Collectors ;
23
28
import org .apache .flink .table .api .DataTypes ;
29
+ import org .apache .flink .table .api .ValidationException ;
24
30
import org .apache .flink .table .catalog .Column ;
25
31
import org .apache .flink .table .catalog .ResolvedSchema ;
32
+ import org .apache .flink .table .catalog .UniqueConstraint ;
26
33
import org .apache .flink .table .legacy .api .TableSchema ;
27
34
import org .apache .flink .table .types .logical .LogicalType ;
28
35
import org .apache .flink .table .types .logical .RowType ;
@@ -59,7 +66,7 @@ public class FlinkSchemaUtil {
59
66
private FlinkSchemaUtil () {}
60
67
61
68
/**
62
- * @deprecated Use {@link #convert(ResolvedSchema)} instead.
69
+ * @deprecated will be removed in 2.0.0; use {@link #convert(ResolvedSchema)} instead.
63
70
*/
64
71
@ Deprecated
65
72
public static Schema convert (TableSchema schema ) {
@@ -102,11 +109,10 @@ public static Schema convert(ResolvedSchema flinkSchema) {
102
109
RowType root = (RowType ) schemaType ;
103
110
Type converted = root .accept (new FlinkTypeToType (root ));
104
111
Schema icebergSchema = new Schema (converted .asStructType ().fields ());
105
- if (flinkSchema .getPrimaryKey ().isPresent ()) {
106
- return freshIdentifierFieldIds (icebergSchema , flinkSchema .getPrimaryKey ().get ().getColumns ());
107
- } else {
108
- return icebergSchema ;
109
- }
112
+ return flinkSchema
113
+ .getPrimaryKey ()
114
+ .map (pk -> freshIdentifierFieldIds (icebergSchema , pk .getColumns ()))
115
+ .orElse (icebergSchema );
110
116
}
111
117
112
118
private static Schema freshIdentifierFieldIds (Schema icebergSchema , List <String > primaryKeys ) {
@@ -137,7 +143,10 @@ private static Schema freshIdentifierFieldIds(Schema icebergSchema, List<String>
137
143
* @param flinkSchema a Flink TableSchema
138
144
* @return the equivalent Schema
139
145
* @throws IllegalArgumentException if the type cannot be converted or there are missing ids
146
+ * @deprecated since 1.10.0, will be removed in 2.0.0. Use {@link #convert(Schema,
147
+ * ResolvedSchema)} instead.
140
148
*/
149
+ @ Deprecated
141
150
public static Schema convert (Schema baseSchema , TableSchema flinkSchema ) {
142
151
// convert to a type with fresh ids
143
152
Types .StructType struct = convert (flinkSchema ).asStruct ();
@@ -155,6 +164,35 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
155
164
}
156
165
}
157
166
167
+ /**
168
+ * Convert a Flink {@link ResolvedSchema} to a {@link Schema} based on the given schema.
169
+ *
170
+ * <p>This conversion does not assign new ids; it uses ids from the base schema.
171
+ *
172
+ * <p>Data types, field order, and nullability will match the Flink type. This conversion may
173
+ * return a schema that is not compatible with base schema.
174
+ *
175
+ * @param baseSchema a Schema on which conversion is based
176
+ * @param flinkSchema a Flink ResolvedSchema
177
+ * @return the equivalent Schema
178
+ * @throws IllegalArgumentException if the type cannot be converted or there are missing ids
179
+ */
180
+ public static Schema convert (Schema baseSchema , ResolvedSchema flinkSchema ) {
181
+ // convert to a type with fresh ids
182
+ Types .StructType struct = convert (flinkSchema ).asStruct ();
183
+ // reassign ids to match the base schema
184
+ Schema schema = TypeUtil .reassignIds (new Schema (struct .fields ()), baseSchema );
185
+ // reassign doc to match the base schema
186
+ schema = TypeUtil .reassignDoc (schema , baseSchema );
187
+
188
+ // fix types that can't be represented in Flink (UUID)
189
+ Schema fixedSchema = FlinkFixupTypes .fixup (schema , baseSchema );
190
+ return flinkSchema
191
+ .getPrimaryKey ()
192
+ .map (pk -> freshIdentifierFieldIds (fixedSchema , pk .getColumns ()))
193
+ .orElse (fixedSchema );
194
+ }
195
+
158
196
/**
159
197
* Convert a {@link Schema} to a {@link RowType Flink type}.
160
198
*
@@ -192,7 +230,10 @@ public static Type convert(LogicalType flinkType) {
192
230
*
193
231
* @param rowType a RowType
194
232
* @return Flink TableSchema
233
+ * @deprecated since 1.10.0, will be removed in 2.0.0. Use {@link #toResolvedSchema(RowType)}
234
+ * instead
195
235
*/
236
+ @ Deprecated
196
237
public static TableSchema toSchema (RowType rowType ) {
197
238
TableSchema .Builder builder = TableSchema .builder ();
198
239
for (RowType .RowField field : rowType .getFields ()) {
@@ -201,12 +242,31 @@ public static TableSchema toSchema(RowType rowType) {
201
242
return builder .build ();
202
243
}
203
244
245
+ /**
246
+ * Convert a {@link RowType} to a {@link ResolvedSchema}.
247
+ *
248
+ * @param rowType a RowType
249
+ * @return Flink ResolvedSchema
250
+ */
251
+ public static ResolvedSchema toResolvedSchema (RowType rowType ) {
252
+ List <Column > columns = Lists .newArrayListWithExpectedSize (rowType .getFieldCount ());
253
+ for (RowType .RowField field : rowType .getFields ()) {
254
+ columns .add (
255
+ Column .physical (field .getName (), TypeConversions .fromLogicalToDataType (field .getType ())));
256
+ }
257
+
258
+ return ResolvedSchema .of (columns );
259
+ }
260
+
204
261
/**
205
262
* Convert a {@link Schema} to a {@link TableSchema}.
206
263
*
207
264
* @param schema iceberg schema to convert.
208
265
* @return Flink TableSchema.
266
+ * @deprecated since 1.10.0, will be removed in 2.0.0. Use {@link #toResolvedSchema(Schema)}
267
+ * instead
209
268
*/
269
+ @ Deprecated
210
270
public static TableSchema toSchema (Schema schema ) {
211
271
TableSchema .Builder builder = TableSchema .builder ();
212
272
@@ -231,4 +291,90 @@ public static TableSchema toSchema(Schema schema) {
231
291
232
292
return builder .build ();
233
293
}
294
+
295
+ /**
296
+ * Convert a {@link Schema} to a {@link ResolvedSchema}.
297
+ *
298
+ * @param schema iceberg schema to convert.
299
+ * @return Flink ResolvedSchema.
300
+ */
301
+ public static ResolvedSchema toResolvedSchema (Schema schema ) {
302
+ RowType rowType = convert (schema );
303
+ List <Column > columns = Lists .newArrayListWithExpectedSize (rowType .getFieldCount ());
304
+
305
+ // Add columns.
306
+ for (RowType .RowField field : rowType .getFields ()) {
307
+ columns .add (
308
+ Column .physical (field .getName (), TypeConversions .fromLogicalToDataType (field .getType ())));
309
+ }
310
+
311
+ // Add primary key.
312
+ Set <Integer > identifierFieldIds = schema .identifierFieldIds ();
313
+ UniqueConstraint uniqueConstraint = null ;
314
+ if (!identifierFieldIds .isEmpty ()) {
315
+ List <String > primaryKeyColumns =
316
+ Lists .newArrayListWithExpectedSize (identifierFieldIds .size ());
317
+ for (Integer identifierFieldId : identifierFieldIds ) {
318
+ String columnName = schema .findColumnName (identifierFieldId );
319
+ Preconditions .checkNotNull (
320
+ columnName , "Cannot find field with id %s in schema %s" , identifierFieldId , schema );
321
+
322
+ primaryKeyColumns .add (columnName );
323
+ }
324
+
325
+ uniqueConstraint =
326
+ UniqueConstraint .primaryKey (UUID .randomUUID ().toString (), primaryKeyColumns );
327
+
328
+ validatePrimaryKey (uniqueConstraint , columns );
329
+ }
330
+
331
+ return new ResolvedSchema (columns , Collections .emptyList (), uniqueConstraint );
332
+ }
333
+
334
+ /**
335
+ * Copied from
336
+ * org.apache.flink.table.catalog.DefaultSchemaResolver#validatePrimaryKey(org.apache.flink.table.catalog.UniqueConstraint,
337
+ * java.util.List)
338
+ */
339
+ private static void validatePrimaryKey (UniqueConstraint primaryKey , List <Column > columns ) {
340
+ final Map <String , Column > columnsByNameLookup =
341
+ columns .stream ().collect (Collectors .toMap (Column ::getName , Function .identity ()));
342
+
343
+ final Set <String > duplicateColumns =
344
+ primaryKey .getColumns ().stream ()
345
+ .filter (name -> Collections .frequency (primaryKey .getColumns (), name ) > 1 )
346
+ .collect (Collectors .toSet ());
347
+
348
+ if (!duplicateColumns .isEmpty ()) {
349
+ throw new ValidationException (
350
+ String .format (
351
+ "Invalid primary key '%s'. A primary key must not contain duplicate columns. Found: %s" ,
352
+ primaryKey .getName (), duplicateColumns ));
353
+ }
354
+
355
+ for (String columnName : primaryKey .getColumns ()) {
356
+ Column column = columnsByNameLookup .get (columnName );
357
+ if (column == null ) {
358
+ throw new ValidationException (
359
+ String .format (
360
+ "Invalid primary key '%s'. Column '%s' does not exist." ,
361
+ primaryKey .getName (), columnName ));
362
+ }
363
+
364
+ if (!column .isPhysical ()) {
365
+ throw new ValidationException (
366
+ String .format (
367
+ "Invalid primary key '%s'. Column '%s' is not a physical column." ,
368
+ primaryKey .getName (), columnName ));
369
+ }
370
+
371
+ final LogicalType columnType = column .getDataType ().getLogicalType ();
372
+ if (columnType .isNullable ()) {
373
+ throw new ValidationException (
374
+ String .format (
375
+ "Invalid primary key '%s'. Column '%s' is nullable." ,
376
+ primaryKey .getName (), columnName ));
377
+ }
378
+ }
379
+ }
234
380
}
0 commit comments