mxm commented on code in PR #13032: URL: https://github.com/apache/iceberg/pull/13032#discussion_r2123765806
########## flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java: ########## @@ -0,0 +1,264 @@ +/* + * 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.sink.dynamic; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TableMetadataCache is responsible for caching table metadata to avoid hitting the catalog too + * frequently. We store table identifier, schema, partition spec, and a set of past schema + * comparison results of the active table schema against the last input schemas. + */ +@Internal +class TableMetadataCache { + + private static final Logger LOG = LoggerFactory.getLogger(TableMetadataCache.class); + private static final int MAX_SIZE = 10; Review Comment: Renamed. ########## flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java: ########## @@ -0,0 +1,266 @@ +/* + * 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.sink.dynamic; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.schema.SchemaWithPartnerVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * Visitor class which compares an input schema to a table schema and emits a compatibility {@link + * Result}. + * + * <ul> + * <li>SAME: The two schemas are semantically identical + * <li>DATA_ADAPTION_NEEDED: We can evolve the data associated with the input schema to match the + * table schema. + * <li>SCHEMA_UPDATE_NEEDED: We need to migrate the table schema to match the input schema. + * </ul> + * + * The input schema fields are compared to the table schema via their names. + */ +public class CompareSchemasVisitor + extends SchemaWithPartnerVisitor<Integer, CompareSchemasVisitor.Result> { + + private final Schema tableSchema; + + private CompareSchemasVisitor(Schema tableSchema) { + this.tableSchema = tableSchema; + } + + public static Result visit(Schema dataSchema, Schema tableSchema) { + return visit(dataSchema, tableSchema, true); + } + + public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { + return visit( + dataSchema, + -1, + new CompareSchemasVisitor(tableSchema), + new PartnerIdByNameAccessors(tableSchema, caseSensitive)); + } + + @Override + public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream) { + if (tableSchemaId == null) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + return downstream; + } + + @Override + public Result struct(Types.StructType struct, Integer tableSchemaId, List<Result> fields) { + if (tableSchemaId == null) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + Result result = fields.stream().reduce(Result::merge).orElse(Result.SCHEMA_UPDATE_NEEDED); + + if (result == Result.SCHEMA_UPDATE_NEEDED) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + Type tableSchemaType = + tableSchemaId == -1 ? tableSchema.asStruct() : tableSchema.findField(tableSchemaId).type(); + if (!tableSchemaType.isStructType()) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + if (struct.fields().size() != tableSchemaType.asStructType().fields().size()) { + return Result.DATA_ADAPTION_NEEDED; + } + + for (int i = 0; i < struct.fields().size(); ++i) { + if (!struct + .fields() + .get(i) + .name() + .equals(tableSchemaType.asStructType().fields().get(i).name())) { + return Result.DATA_ADAPTION_NEEDED; + } + } + + return result; + } + + @Override + public Result field(Types.NestedField field, Integer tableSchemaId, Result typeResult) { + if (tableSchemaId == null) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + if (typeResult != Result.SAME) { + return typeResult; + } + + if (tableSchema.findField(tableSchemaId).isRequired() && field.isOptional()) { + return Result.SCHEMA_UPDATE_NEEDED; + } else { + return Result.SAME; + } + } + + @Override + public Result list(Types.ListType list, Integer tableSchemaId, Result elementsResult) { + if (tableSchemaId == null) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + return elementsResult; + } + + @Override + public Result map( + Types.MapType map, Integer tableSchemaId, Result keyResult, Result valueResult) { + if (tableSchemaId == null) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + return keyResult.merge(valueResult); + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Result primitive(Type.PrimitiveType primitive, Integer tableSchemaId) { + if (tableSchemaId == null) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + Type tableSchemaType = tableSchema.findField(tableSchemaId).type(); + if (!tableSchemaType.isPrimitiveType()) { + return Result.SCHEMA_UPDATE_NEEDED; + } + + Type.PrimitiveType tableSchemaPrimitiveType = tableSchemaType.asPrimitiveType(); + if (primitive.equals(tableSchemaPrimitiveType)) { + return Result.SAME; + } else if (primitive.equals(Types.IntegerType.get()) + && tableSchemaPrimitiveType.equals(Types.LongType.get())) { + return Result.DATA_ADAPTION_NEEDED; + } else if (primitive.equals(Types.FloatType.get()) + && tableSchemaPrimitiveType.equals(Types.DoubleType.get())) { + return Result.DATA_ADAPTION_NEEDED; + } else if (primitive.equals(Types.DateType.get()) + && tableSchemaPrimitiveType.equals(Types.TimestampType.withoutZone())) { + return Result.DATA_ADAPTION_NEEDED; + } else if (primitive.typeId() == Type.TypeID.DECIMAL + && tableSchemaPrimitiveType.typeId() == Type.TypeID.DECIMAL) { + Types.DecimalType dataType = (Types.DecimalType) primitive; + Types.DecimalType tableType = (Types.DecimalType) tableSchemaPrimitiveType; + return dataType.scale() == tableType.scale() && dataType.precision() < tableType.precision() + ? Result.DATA_ADAPTION_NEEDED + : Result.SCHEMA_UPDATE_NEEDED; + } else { + return Result.SCHEMA_UPDATE_NEEDED; + } + } + + static class PartnerIdByNameAccessors implements PartnerAccessors<Integer> { + private final Schema tableSchema; + private boolean caseSensitive = true; + + PartnerIdByNameAccessors(Schema tableSchema) { + this.tableSchema = tableSchema; + } + + private PartnerIdByNameAccessors(Schema tableSchema, boolean caseSensitive) { + this(tableSchema); + this.caseSensitive = caseSensitive; + } + + @Override + public Integer fieldPartner(Integer tableSchemaFieldId, int fieldId, String name) { + Types.StructType struct; + if (tableSchemaFieldId == -1) { + struct = tableSchema.asStruct(); + } else { + struct = tableSchema.findField(tableSchemaFieldId).type().asStructType(); + } + + Types.NestedField field = + caseSensitive ? struct.field(name) : struct.caseInsensitiveField(name); + if (field != null) { + return field.fieldId(); + } + + return null; + } + + @Override + public Integer mapKeyPartner(Integer tableSchemaMapId) { + Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); + if (mapField != null) { + return mapField.type().asMapType().fields().get(0).fieldId(); + } + + return null; + } + + @Override + public Integer mapValuePartner(Integer tableSchemaMapId) { + Types.NestedField mapField = tableSchema.findField(tableSchemaMapId); + if (mapField != null) { + return mapField.type().asMapType().fields().get(1).fieldId(); + } + + return null; + } + + @Override + public Integer listElementPartner(Integer tableSchemaListId) { + Types.NestedField listField = tableSchema.findField(tableSchemaListId); + if (listField != null) { + return listField.type().asListType().fields().get(0).fieldId(); + } + + return null; + } + } + + public enum Result { + SAME(0), + DATA_ADAPTION_NEEDED(1), Review Comment: Renamed. -- 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. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org