danny0405 commented on code in PR #5443:
URL: https://github.com/apache/hudi/pull/5443#discussion_r869853961


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/SchemaEvoContext.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.hudi.table.format;
+
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.internal.schema.InternalSchema;
+
+import java.io.Serializable;
+
+/**
+ * Data class to pass schema evolution info from table source to input format.
+ */
+public final class SchemaEvoContext implements Serializable {
+  private final boolean enabled;

Review Comment:
   Is this clazz necessary ? The `enabled` flag can be replaced by Option< 
querySchema> non empty instead.



##########
hudi-flink-datasource/hudi-flink/pom.xml:
##########
@@ -265,6 +265,64 @@
 
         <!-- Test dependencies -->
 
+        <!-- Spark test -->
+        <!-- Spark testkit is used to prepare test data for schema evolution 
(with changed types, renamed columns, and so on). -->
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.eclipse.jetty.orbit</groupId>
+                    <artifactId>javax.servlet</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>javax.servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.twitter</groupId>
+                    <artifactId>chill-java</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-hive_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>

Review Comment:
   Why introduces the spark dependency in flink pom ? 



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java:
##########
@@ -447,6 +453,17 @@ private Schema inferSchemaFromDdl() {
     return HoodieAvroUtils.addMetadataFields(schema, 
conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
   }
 
+  private SchemaEvoContext getSchemaEvoContext() {
+    if (!conf.getBoolean(FlinkOptions.SCHEMA_EVOLUTION_ENABLED)) {

Review Comment:
   Returns `Option<SchemaEvoContext>` instead.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java:
##########
@@ -390,4 +417,70 @@ private InflaterInputStreamFactory<?> 
getInflaterInputStreamFactory(org.apache.h
       return null;
     }
   }
+
+  private ActualFields prepareSchemaEvo(FileInputSplit fileSplit) {
+    if (!schemaEvoContext.isEnabled()) {
+      projection = null;
+      return new ActualFields(fullFieldNames, fullFieldTypes);
+    }
+    InternalSchema mergedSchema = getMergedSchema(fileSplit);
+    ActualFields actualFields = getActualFields(mergedSchema);
+    projection = getProjection(mergedSchema, actualFields);
+    return actualFields;
+  }
+
+  private InternalSchema getMergedSchema(FileInputSplit fileSplit) {
+    long commitTime = 
Long.parseLong(FSUtils.getCommitTime(fileSplit.getPath().getPath()));
+    InternalSchema fileSchema = 
InternalSchemaCache.searchSchemaAndCache(commitTime, 
schemaEvoContext.metaClient(), false);
+    InternalSchema querySchema = schemaEvoContext.querySchema();
+    return new InternalSchemaMerger(fileSchema, querySchema, true, 
true).mergeSchema();
+  }
+
+  private ActualFields getActualFields(InternalSchema mergedSchema) {
+    int skipFields = HOODIE_META_COLUMNS.size();
+    String[] actualFieldNames = mergedSchema.columns()
+        .stream()
+        .skip(skipFields)
+        .map(Types.Field::name)
+        .toArray(String[]::new);
+    Schema actualSchema = AvroInternalSchemaConverter.convert(mergedSchema, 
schemaEvoContext.tableName());
+    DataType[] actualFieldTypes = 
AvroSchemaConverter.convertToDataType(actualSchema).getChildren()
+        .stream()
+        .skip(skipFields)
+        .toArray(DataType[]::new);
+    return new ActualFields(actualFieldNames, actualFieldTypes);
+  }
+
+  private RowDataProjection getProjection(InternalSchema mergedSchema, 
ActualFields actualFields) {
+    CastMap castMap = CastMap.of(schemaEvoContext.tableName(), 
schemaEvoContext.querySchema(), mergedSchema);
+    if (castMap.containsAnyPos(selectedFields)) {
+      LogicalType[] types = 
Arrays.stream(actualFields.types()).map(DataType::getLogicalType).toArray(LogicalType[]::new);
+      LogicalType[] readType = new LogicalType[selectedFields.length];
+      for (int i = 0; i < selectedFields.length; i++) {
+        readType[i] = types[selectedFields[i]];
+      }
+      int[] pos = IntStream.range(0, selectedFields.length).toArray();
+      return RowDataProjection.instance(RowType.of(readType), pos, 
castMap.rearrange(selectedFields, pos));
+    } else {
+      return null;
+    }
+  }
+
+  private static final class ActualFields {
+    private final String[] names;

Review Comment:
   Personally i don't like the style that we introduces too many intermediate 
POJOs.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/CastMap.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.hudi.table.format;
+
+import org.apache.avro.Schema;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.util.Preconditions;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.Type;
+import org.apache.hudi.internal.schema.Types;
+import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
+import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
+import org.apache.hudi.util.AvroSchemaConverter;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BIGINT;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DATE;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DOUBLE;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.FLOAT;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTEGER;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.VARCHAR;
+import static org.apache.hudi.common.model.HoodieRecord.HOODIE_META_COLUMNS;
+
+/**
+ * CastMap is responsible for type conversion when full schema evolution 
enabled.
+ */
+public final class CastMap {
+  // Maps position (column number) to corresponding cast
+  private final Map<Integer, Cast> castMap = new HashMap<>();
+
+  /**
+   * Creates CastMap by comparing two schemes. Cast of a specific column is 
created if its type has changed.
+   */
+  public static CastMap of(String tableName, InternalSchema querySchema, 
InternalSchema actualSchema) {
+    DataType queryType = internalSchemaToDataType(tableName, querySchema);
+    DataType actualType = internalSchemaToDataType(tableName, actualSchema);
+    int metaColumnsSize = HOODIE_META_COLUMNS.size();
+    CastMap castMap = new CastMap();
+    InternalSchemaUtils.collectTypeChangedCols(querySchema, 
actualSchema).entrySet()
+            .stream()
+            .filter(e -> e.getKey() >= metaColumnsSize)
+            .filter(e -> !isSameType(e.getValue().getLeft(), 
e.getValue().getRight()))
+            .forEach(e -> {
+              int pos = e.getKey();
+              LogicalType target = 
queryType.getChildren().get(pos).getLogicalType();
+              LogicalType actual = 
actualType.getChildren().get(pos).getLogicalType();
+              castMap.add(pos - metaColumnsSize, actual, target);
+            });
+    return castMap;
+  }
+
+  public Object castIfNeed(int pos, Object val) {
+    Cast cast = castMap.get(pos);
+    if (cast == null) {
+      return val;
+    }
+    return cast(val, cast.from(), cast.to());
+  }
+
+  private Object cast(Object val, LogicalType fromType, LogicalType toType) {
+    LogicalTypeRoot from = fromType.getTypeRoot();
+    LogicalTypeRoot to = toType.getTypeRoot();
+    switch (to) {
+      case BIGINT: {
+        // Integer => Long

Review Comment:
   What is the philosophy of these mappings ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataProjection.java:
##########
@@ -61,7 +68,10 @@ public static RowDataProjection instance(LogicalType[] 
types, int[] positions) {
   public RowData project(RowData rowData) {
     GenericRowData genericRowData = new 
GenericRowData(this.fieldGetters.length);
     for (int i = 0; i < this.fieldGetters.length; i++) {

Review Comment:
   Can we do not affect the normal code path for non evolution ? Something like
   ```java
   public RowData project(RowData rowData, CastMap castMap)
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to