You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by da...@apache.org on 2023/02/27 02:29:05 UTC

[hudi] branch master updated: [HUDI-5845] Remove usage of deprecated getTableAvroSchemaWithoutMetadataFields. (#8032)

This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 5af78c665bd [HUDI-5845] Remove usage of deprecated getTableAvroSchemaWithoutMetadataFields. (#8032)
5af78c665bd is described below

commit 5af78c665bd8261269c612832a7d3fefe2957217
Author: slfan1989 <55...@users.noreply.github.com>
AuthorDate: Mon Feb 27 10:28:57 2023 +0800

    [HUDI-5845] Remove usage of deprecated getTableAvroSchemaWithoutMetadataFields. (#8032)
    
    Co-authored-by: slfan1989 <louj1988@@>
---
 .../src/main/java/org/apache/hudi/table/HoodieTable.java              | 2 +-
 .../src/main/scala/org/apache/hudi/HoodieCLIUtils.scala               | 2 +-
 .../src/main/scala/org/apache/hudi/IncrementalRelation.scala          | 4 ++--
 .../spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala  | 2 +-
 .../src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala     | 2 +-
 .../src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala | 2 +-
 .../apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java  | 2 +-
 7 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
index 4102515ae01..2a71cf4ea46 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
@@ -814,7 +814,7 @@ public abstract class HoodieTable<T, I, K, O> implements Serializable {
     try {
       TableSchemaResolver schemaResolver = new TableSchemaResolver(getMetaClient());
       writerSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema());
-      tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaResolver.getTableAvroSchemaWithoutMetadataFields());
+      tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaResolver.getTableAvroSchema(false));
       isValid = isSchemaCompatible(tableSchema, writerSchema, config.shouldAllowAutoEvolutionColumnDrop());
     } catch (Exception e) {
       throw new HoodieException("Failed to read schema/check compatibility for base path " + metaClient.getBasePath(), e);
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala
index 0d3edd592d1..d3ead551089 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala
@@ -39,7 +39,7 @@ object HoodieCLIUtils {
     val metaClient = HoodieTableMetaClient.builder().setBasePath(basePath)
       .setConf(sparkSession.sessionState.newHadoopConf()).build()
     val schemaUtil = new TableSchemaResolver(metaClient)
-    val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
+    val schemaStr = schemaUtil.getTableAvroSchema(false).toString
     val finalParameters = HoodieWriterUtils.parametersWithWriteDefaults(
       withSparkConf(sparkSession, Map.empty)(
         conf + (DataSourceWriteOptions.TABLE_TYPE.key() -> metaClient.getTableType.name()))
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala
index 80ee3dde5b0..b4ca19c6e0f 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala
@@ -101,10 +101,10 @@ class IncrementalRelation(val sqlContext: SQLContext,
     }
 
     val tableSchema = if (useEndInstantSchema && iSchema.isEmptySchema) {
-      if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchemaWithoutMetadataFields() else
+      if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchema(false) else
         schemaResolver.getTableAvroSchema(commitsToReturn.last, false)
     } else {
-      schemaResolver.getTableAvroSchemaWithoutMetadataFields()
+      schemaResolver.getTableAvroSchema(false)
     }
     if (tableSchema.getType == Schema.Type.NULL) {
       // if there is only one commit in the table and is an empty commit without schema, return empty RDD here
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
index cb5a3f6fa75..c4763bbe266 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
@@ -96,7 +96,7 @@ case class AlterHoodieTableChangeColumnCommand(
 
   private def validateSchema(newSchema: Schema, metaClient: HoodieTableMetaClient): Unit = {
     val schemaUtil = new TableSchemaResolver(metaClient)
-    val tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchemaWithoutMetadataFields)
+    val tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchema(false))
     if (!AvroSchemaUtils.isSchemaCompatible(tableSchema, newSchema, true)) {
       throw new HoodieException("Failed schema compatibility check for newSchema :" + newSchema +
         ", origin table schema :" + tableSchema + ", base path :" + metaClient.getBasePath)
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
index 28924616e3b..72dbb0c2539 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
@@ -1226,7 +1226,7 @@ class TestHoodieSparkSqlWriter {
       .setConf(spark.sparkContext.hadoopConfiguration)
       .setBasePath(tempBasePath)
       .build()
-    new TableSchemaResolver(tableMetaClient).getTableAvroSchemaWithoutMetadataFields
+    new TableSchemaResolver(tableMetaClient).getTableAvroSchema(false)
   }
 }
 
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala
index 76e8c26b7bd..d681cd7a766 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala
@@ -259,7 +259,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
     }
 
     val tableMetaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath).build()
-    val actualSchema = new TableSchemaResolver(tableMetaClient).getTableAvroSchemaWithoutMetadataFields
+    val actualSchema = new TableSchemaResolver(tableMetaClient).getTableAvroSchema(false)
     val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(commonOpts(HoodieWriteConfig.TBL_NAME.key))
     spark.sparkContext.getConf.registerKryoClasses(
       Array(classOf[org.apache.avro.generic.GenericData],
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java
index f85e55dfd40..7932d839a5f 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java
@@ -778,7 +778,7 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
     assertEquals(1900, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
 
     TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(fs.getConf()).build());
-    Schema tableSchema = tableSchemaResolver.getTableAvroSchemaWithoutMetadataFields();
+    Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false);
     assertNotNull(tableSchema);
 
     Schema expectedSchema;