[HUDI-284] add more test for UpdateSchemaEvolution (#2127)

Unit test different schema evolution scenarios.
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
index 77fef5c..faa7ff6 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
@@ -197,7 +197,6 @@
       } else {
         recordsDeleted++;
       }
-
       writeStatus.markSuccess(hoodieRecord, recordMetadata);
       // deflate record payload after recording success. This will help users access payload as a
       // part of marking
@@ -243,16 +242,14 @@
     if (copyOldRecord) {
       // this should work as it is, since this is an existing record
       String errMsg = "Failed to merge old record into new file for key " + key + " from old file " + getOldFilePath()
-          + " to new file " + newFilePath;
+          + " to new file " + newFilePath + " with writerSchema " + writerSchemaWithMetafields.toString(true);
       try {
         fileWriter.writeAvro(key, oldRecord);
       } catch (ClassCastException e) {
-        LOG.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + getOldFilePath()
-            + " to file " + newFilePath + " with writerSchema " + writerSchemaWithMetafields.toString(true));
+        LOG.debug("Old record is " + oldRecord);
         throw new HoodieUpsertException(errMsg, e);
-      } catch (IOException e) {
-        LOG.error("Failed to merge old record into new file for key " + key + " from old file " + getOldFilePath()
-            + " to new file " + newFilePath, e);
+      } catch (IOException | RuntimeException e) {
+        LOG.debug("Old record is " + oldRecord);
         throw new HoodieUpsertException(errMsg, e);
       }
       recordsWritten++;
diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.txt b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.avsc
similarity index 100%
rename from hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.txt
rename to hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.avsc
diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaChangeOrder.avsc b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaChangeOrder.avsc
new file mode 100644
index 0000000..16844ff
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaChangeOrder.avsc
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+{
+    "namespace": "example.schema",
+    "type": "record",
+    "name": "trip",
+    "fields": [
+        {
+            "name": "time",
+            "type": "string"
+        },
+        {
+            "name": "_row_key",
+            "type": "string"
+        },
+        {
+            "name": "added_field",
+            "type": ["int", "null"]
+        },
+        {
+             "name": "number",
+             "type": ["int", "null"]
+        }
+    ]
+}
diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaColumnRequire.avsc b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaColumnRequire.avsc
new file mode 100644
index 0000000..3c9807c
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaColumnRequire.avsc
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+{
+    "namespace": "example.schema",
+    "type": "record",
+    "name": "trip",
+    "fields": [
+        {
+            "name": "number",
+            "type": ["int", "null"]
+        },
+        {
+            "name": "time",
+            "type": "string"
+        },
+        {
+            "name": "_row_key",
+            "type": "string"
+        },
+        {
+            "name": "added_field",
+            "type": "int"
+        }
+    ]
+}
diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaColumnType.avsc b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaColumnType.avsc
new file mode 100644
index 0000000..cf04b54
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaColumnType.avsc
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+{
+    "namespace": "example.schema",
+    "type": "record",
+    "name": "trip",
+    "fields": [
+        {
+            "name": "number",
+            "type": ["string", "null"]
+        },
+        {
+            "name": "time",
+            "type": "string"
+        },
+        {
+            "name": "_row_key",
+            "type": "string"
+        }
+    ]
+}
diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaDeleteColumn.avsc b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaDeleteColumn.avsc
new file mode 100644
index 0000000..797dadc
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchemaDeleteColumn.avsc
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+{
+    "namespace": "example.schema",
+    "type": "record",
+    "name": "trip",
+    "fields": [
+        {
+            "name": "time",
+            "type": "string"
+        },
+        {
+            "name": "_row_key",
+            "type": "string"
+        }
+    ]
+}
diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleSchema.txt b/hudi-client/hudi-client-common/src/test/resources/exampleSchema.avsc
similarity index 100%
rename from hudi-client/hudi-client-common/src/test/resources/exampleSchema.txt
rename to hudi-client/hudi-client-common/src/test/resources/exampleSchema.avsc
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
index 8058304..9a8d7e0 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
@@ -27,6 +27,7 @@
 import org.apache.hudi.common.testutils.RawTripTestPayload;
 import org.apache.hudi.common.util.ParquetUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieUpsertException;
 import org.apache.hudi.io.HoodieCreateHandle;
 import org.apache.hudi.io.HoodieMergeHandle;
 import org.apache.hudi.table.HoodieSparkTable;
@@ -34,12 +35,14 @@
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.avro.AvroReadSupport;
+import org.apache.parquet.io.InvalidRecordException;
+import org.apache.parquet.io.ParquetDecodingException;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.function.Executable;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -51,7 +54,7 @@
 
 import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 
 public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
 
@@ -68,77 +71,158 @@
     cleanupResources();
   }
 
-  @Test
-  public void testSchemaEvolutionOnUpdate() throws Exception {
-    // Create a bunch of records with a old version of schema
-    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
+  private WriteStatus prepareFirstRecordCommit(List<String> recordsStrs) throws IOException {
+    // Create a bunch of records with an old version of schema
+    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.avsc");
     final HoodieSparkTable table = HoodieSparkTable.create(config, context);
-
     final List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
-      String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
-          + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
-      String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
-          + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
-      String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
-          + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
       List<HoodieRecord> insertRecords = new ArrayList<>();
-      RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
-      insertRecords
-          .add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
-      RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
-      insertRecords
-          .add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
-      RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
-      insertRecords
-          .add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
-
+      for (String recordStr : recordsStrs) {
+        RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
+        insertRecords
+            .add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
+      }
       Map<String, HoodieRecord> insertRecordMap = insertRecords.stream()
           .collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity()));
       HoodieCreateHandle createHandle =
-          new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecordMap, supplier);
+          new HoodieCreateHandle(config, "100", table, insertRecords.get(0).getPartitionPath(), "f1-0", insertRecordMap, supplier);
       createHandle.write();
       return createHandle.close();
     }).collect();
 
     final Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
     FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile);
+    return statuses.get(0);
+  }
 
-    // Now try an update with an evolved schema
-    // Evolved schema does not have guarantee on preserving the original field ordering
-    final HoodieWriteConfig config2 = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
-    final WriteStatus insertResult = statuses.get(0);
-    String fileId = insertResult.getFileId();
+  private List<String> generateMultipleRecordsForExampleSchema() {
+    List<String> recordsStrs = new ArrayList<>();
+    String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+    String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+    String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+    recordsStrs.add(recordStr1);
+    recordsStrs.add(recordStr2);
+    recordsStrs.add(recordStr3);
+    return recordsStrs;
+  }
 
-    final HoodieSparkTable table2 = HoodieSparkTable.create(config, context);
-    assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> {
-      // New content with values for the newly added field
-      String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
-          + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
-      List<HoodieRecord> updateRecords = new ArrayList<>();
-      RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
-      HoodieRecord record1 =
-          new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
-      record1.unseal();
-      record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
-      record1.seal();
-      updateRecords.add(record1);
+  private List<String> generateOneRecordForExampleSchema() {
+    List<String> recordsStrs = new ArrayList<>();
+    String recordStr = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+    recordsStrs.add(recordStr);
+    return recordsStrs;
+  }
 
-      assertDoesNotThrow(() -> {
-        HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
-            updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier);
-        Configuration conf = new Configuration();
-        AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchemaWithMetafields());
-        List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
-            new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath()));
+  private void assertSchemaEvolutionOnUpdateResult(WriteStatus insertResult, HoodieSparkTable updateTable,
+                                                   List<HoodieRecord> updateRecords, String assertMsg, boolean isAssertThrow, Class expectedExceptionType) {
+    jsc.parallelize(Arrays.asList(1)).map(x -> {
+      Executable executable = () -> {
+        HoodieMergeHandle mergeHandle = new HoodieMergeHandle(updateTable.getConfig(), "101", updateTable,
+            updateRecords.iterator(), updateRecords.get(0).getPartitionPath(), insertResult.getFileId(), supplier);
+        AvroReadSupport.setAvroReadSchema(updateTable.getHadoopConf(), mergeHandle.getWriterSchemaWithMetafields());
+        List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(updateTable.getHadoopConf(),
+            new Path(updateTable.getConfig().getBasePath() + "/" + insertResult.getStat().getPath()));
         for (GenericRecord rec : oldRecords) {
           mergeHandle.write(rec);
         }
         mergeHandle.close();
-      }, "UpdateFunction could not read records written with exampleSchema.txt using the "
-          + "exampleEvolvedSchema.txt");
-
+      };
+      if (isAssertThrow) {
+        assertThrows(expectedExceptionType, executable, assertMsg);
+      } else {
+        assertDoesNotThrow(executable, assertMsg);
+      }
       return 1;
-    }).collect().size());
+    }).collect();
+  }
+
+  private List<HoodieRecord> buildUpdateRecords(String recordStr, String insertFileId) throws IOException {
+    List<HoodieRecord> updateRecords = new ArrayList<>();
+    RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
+    HoodieRecord record =
+        new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange);
+    record.setCurrentLocation(new HoodieRecordLocation("101", insertFileId));
+    record.seal();
+    updateRecords.add(record);
+    return updateRecords;
+  }
+
+  @Test
+  public void testSchemaEvolutionOnUpdateSuccessWithAddColumnHaveDefault() throws Exception {
+    final WriteStatus insertResult = prepareFirstRecordCommit(generateMultipleRecordsForExampleSchema());
+    // Now try an update with an evolved schema
+    // Evolved schema does not have guarantee on preserving the original field ordering
+    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleEvolvedSchema.avsc");
+    final HoodieSparkTable table = HoodieSparkTable.create(config, context);
+    // New content with values for the newly added field
+    String recordStr = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
+    List<HoodieRecord> updateRecords = buildUpdateRecords(recordStr, insertResult.getFileId());
+    String assertMsg = "UpdateFunction could not read records written with exampleSchema.avsc using the "
+        + "exampleEvolvedSchema.avsc";
+    assertSchemaEvolutionOnUpdateResult(insertResult, table, updateRecords, assertMsg, false, null);
+  }
+
+  @Test
+  public void testSchemaEvolutionOnUpdateSuccessWithChangeColumnOrder() throws Exception {
+    final WriteStatus insertResult = prepareFirstRecordCommit(generateMultipleRecordsForExampleSchema());
+    // Now try an update with an evolved schema
+    // Evolved schema does not have guarantee on preserving the original field ordering
+    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleEvolvedSchemaChangeOrder.avsc");
+    final HoodieSparkTable table = HoodieSparkTable.create(config, context);
+    String recordStr = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"added_field\":1},\"number\":12";
+    List<HoodieRecord> updateRecords = buildUpdateRecords(recordStr, insertResult.getFileId());
+    String assertMsg = "UpdateFunction could not read records written with exampleSchema.avsc using the "
+        + "exampleEvolvedSchemaChangeOrder.avsc as column order change";
+    assertSchemaEvolutionOnUpdateResult(insertResult, table, updateRecords, assertMsg, false, null);
+  }
+
+  @Test
+  public void testSchemaEvolutionOnUpdateMisMatchWithDeleteColumn() throws Exception {
+    final WriteStatus insertResult = prepareFirstRecordCommit(generateOneRecordForExampleSchema());
+    // Now try an update with an evolved schema
+    // Evolved schema does not have guarantee on preserving the original field ordering
+    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleEvolvedSchemaDeleteColumn.avsc");
+    final HoodieSparkTable table = HoodieSparkTable.create(config, context);
+    String recordStr = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\"}";
+    List<HoodieRecord> updateRecords = buildUpdateRecords(recordStr, insertResult.getFileId());
+    String assertMsg = "UpdateFunction when delete column, Parquet/Avro schema mismatch: Avro field 'xxx' not found";
+    assertSchemaEvolutionOnUpdateResult(insertResult, table, updateRecords, assertMsg, true, InvalidRecordException.class);
+  }
+
+  @Test
+  public void testSchemaEvolutionOnUpdateMisMatchWithAddColumnNotHaveDefault() throws Exception {
+    final WriteStatus insertResult = prepareFirstRecordCommit(generateOneRecordForExampleSchema());
+    // Now try an update with an evolved schema
+    // Evolved schema does not have guarantee on preserving the original field ordering
+    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleEvolvedSchemaColumnRequire.avsc");
+    final HoodieSparkTable table = HoodieSparkTable.create(config, context);
+    String recordStr = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
+    List<HoodieRecord> updateRecords = buildUpdateRecords(recordStr, insertResult.getFileId());
+    String assertMsg = "UpdateFunction could not read records written with exampleSchema.avsc using the "
+        + "exampleEvolvedSchemaColumnRequire.avsc, because old records do not have required column added_field";
+    assertSchemaEvolutionOnUpdateResult(insertResult, table, updateRecords, assertMsg, true, HoodieUpsertException.class);
+  }
+
+  @Test
+  public void testSchemaEvolutionOnUpdateMisMatchWithChangeColumnType() throws Exception {
+    final WriteStatus insertResult = prepareFirstRecordCommit(generateOneRecordForExampleSchema());
+    // Now try an update with an evolved schema
+    // Evolved schema does not have guarantee on preserving the original field ordering
+    final HoodieWriteConfig config = makeHoodieClientConfig("/exampleEvolvedSchemaColumnType.avsc");
+    final HoodieSparkTable table = HoodieSparkTable.create(config, context);
+    String recordStr = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":\"12\"}";
+    List<HoodieRecord> updateRecords = buildUpdateRecords(recordStr, insertResult.getFileId());
+    String assertMsg = "UpdateFunction when change column type, org.apache.parquet.avro.AvroConverters$FieldUTF8Converter";
+    assertSchemaEvolutionOnUpdateResult(insertResult, table, updateRecords, assertMsg, true, ParquetDecodingException.class);
   }
 
   private HoodieWriteConfig makeHoodieClientConfig(String name) {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
index d3b5867..f10e845 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
@@ -69,7 +69,7 @@
 
 public class TestHoodieIndex extends HoodieClientTestHarness {
 
-  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieIndex.class, "/exampleSchema.txt", true);
+  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieIndex.class, "/exampleSchema.avsc", true);
   private final Random random = new Random();
   private IndexType indexType;
   private HoodieIndex index;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
index 39d9b64..8446698 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
@@ -68,7 +68,7 @@
 
 public class TestHoodieBloomIndex extends HoodieClientTestHarness {
 
-  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.txt", true);
+  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.avsc", true);
   private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
 
   public static Stream<Arguments> configParams() {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
index e6fc3be..c18eeb1 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
@@ -57,7 +57,7 @@
 
 public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
 
-  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.txt", true);
+  private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.avsc", true);
 
   @BeforeEach
   public void setUp() throws Exception {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
index 852f802..c054bc4 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
@@ -81,7 +81,7 @@
 public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
 
   private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteActionExecutor.class);
-  private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/exampleSchema.txt");
+  private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/exampleSchema.avsc");
 
   @Test
   public void testMakeNewPath() {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
index 6b3426b..c19427c 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
@@ -64,7 +64,7 @@
 public class TestUpsertPartitioner extends HoodieClientTestBase {
 
   private static final Logger LOG = LogManager.getLogger(TestUpsertPartitioner.class);
-  private static final Schema SCHEMA = getSchemaFromResource(TestUpsertPartitioner.class, "/exampleSchema.txt");
+  private static final Schema SCHEMA = getSchemaFromResource(TestUpsertPartitioner.class, "/exampleSchema.avsc");
 
   private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, int numUpdates, int fileSize,
       String testPartitionPath, boolean autoSplitInserts) throws Exception {