sivabalan narayanan created HUDI-9203:
-----------------------------------------

             Summary: Fix usability w/ upgrading from 0.x to 1.x and wish to 
stay in table version 6
                 Key: HUDI-9203
                 URL: https://issues.apache.org/jira/browse/HUDI-9203
             Project: Apache Hudi
          Issue Type: Improvement
          Components: writer-core
            Reporter: sivabalan narayanan


When a user upgrades a table from 0.15.0 to 1.x and prefers to stay with table 
version 6, as per the migration guide 
([https://hudi.apache.org/docs/deployment#upgrading-to-100),] we expect the 
user to set only two configs. 

 

{{hoodie.write.auto.upgrade}} to false.

{{hoodie.metadata.enable = false. }}

{{}}

{{{}and w/ latest master, we have added support for b/w compatible writer as 
well. So, that leaves us w/ just 1 config i.e. 
"{}}}{{{}hoodie.write.auto.upgrade{}}}{{{}=false". {}}}

{{}}

{{But when attempted to user 1.0 binary to write to a hudi table created using 
0.x, we error out as below. }}

{{}}
{code:java}
df1.write.format("hudi").
     |   options(getQuickstartWriteConfigs).
     |   option("hoodie.write.auto.upgrade","false").
     |   option(PRECOMBINE_FIELD_OPT_KEY, "ts").
     |   option(RECORDKEY_FIELD_OPT_KEY, "uuid").
     |   option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
     |   option(TABLE_NAME, tableName).
     |   mode(Append).
     |   save(basePath)
warning: one deprecation; for details, enable `:setting -deprecation' or 
`:replay -deprecation'
# WARNING: Unable to attach Serviceability Agent. Unable to attach even with 
module exceptions: [org.apache.hudi.org.openjdk.jol.vm.sa.SASupportException: 
Sense failed., org.apache.hudi.org.openjdk.jol.vm.sa.SASupportException: Sense 
failed., org.apache.hudi.org.openjdk.jol.vm.sa.SASupportException: Sense 
failed.]
org.apache.hudi.exception.HoodieUpgradeDowngradeException: Please disable 
metadata table before upgrading from version SIX to EIGHT.
  at 
org.apache.hudi.table.upgrade.UpgradeDowngrade.needsUpgradeOrDowngrade(UpgradeDowngrade.java:72)
  at 
org.apache.hudi.table.upgrade.UpgradeDowngrade.needsUpgradeOrDowngrade(UpgradeDowngrade.java:80)
  at 
org.apache.hudi.client.BaseHoodieWriteClient.needsUpgradeOrDowngrade(BaseHoodieWriteClient.java:1490)
  at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommitWithTime(BaseHoodieWriteClient.java:956)
  at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommitWithTime(BaseHoodieWriteClient.java:949)
  at 
org.apache.hudi.HoodieSparkSqlWriterInternal.writeInternal(HoodieSparkSqlWriter.scala:514)
  at 
org.apache.hudi.HoodieSparkSqlWriterInternal.$anonfun$write$1(HoodieSparkSqlWriter.scala:192)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:118)
  at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:195)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:103)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:827)
  at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65)
  at 
org.apache.spark.sql.adapter.BaseSpark3Adapter.sqlExecutionWithNewExecutionId(BaseSpark3Adapter.scala:107)
  at 
org.apache.hudi.HoodieSparkSqlWriterInternal.write(HoodieSparkSqlWriter.scala:214)
  at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:129)
  at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:171)
  at 
org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:47)
  at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
  at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
  at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
  at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:118)
  at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:195)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:103)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:827)
  at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65)
  at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
  at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
  at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:512)
  at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
  at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:512)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:31)
  at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
  at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
  at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:488)
  at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:94)
  at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:81)
  at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:79)
  at 
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:133)
  at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:856)
  at 
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:387)
  at 
org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:360)
  at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
  ... 67 elided


scala>  {code}
{{}}

{{}}

{{}}

{{}}

{{script to reproduce: }}

{{Just follow the quick start. }}
{{Using 0.14.1 spark bundle: }}
{code:java}
import org.apache.hudi.QuickstartUtils._
import scala.collection.JavaConversions._
import org.apache.spark.sql.SaveMode._
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.config.HoodieWriteConfig._
import org.apache.hudi.common.model.HoodieRecord

val tableName = "hudi_trips_cow"
val basePath = "file:///tmp/hudi_trips_cow"
val dataGen = new DataGenerator{code}
{code:java}
val inserts = convertToStringList(dataGen.generateInserts(10))
val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
df.write.format("hudi").
  options(getQuickstartWriteConfigs).
  option(PRECOMBINE_FIELD_OPT_KEY, "ts").
  option(RECORDKEY_FIELD_OPT_KEY, "uuid").
  option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
  option(TABLE_NAME, tableName).
  mode(Overwrite).
  save(basePath){code}
{code:java}
 {code}


re-launch spark shell from latest master. 
{code:java}
import org.apache.hudi.QuickstartUtils._
import scala.collection.JavaConversions._
import org.apache.spark.sql.SaveMode._
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.config.HoodieWriteConfig._
import org.apache.hudi.common.model.HoodieRecord

val tableName = "hudi_trips_cow"
val basePath = "file:///tmp/hudi_trips_cow"
val dataGen = new DataGenerator{code}
{code:java}
val df1 = 
spark.read.format("hudi").load(basePath).drop("_hoodie_commit_time","_hoodie_commit_seqno","_hoodie_record_key","_hoodie_partition_path","_hoodie_file_name").limit(2){code}
{code:java}
 df1.write.format("hudi").
     |   options(getQuickstartWriteConfigs).
     |   option("hoodie.write.auto.upgrade","false").
     |   option(PRECOMBINE_FIELD_OPT_KEY, "ts").
     |   option(RECORDKEY_FIELD_OPT_KEY, "uuid").
     |   option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
     |   option(TABLE_NAME, tableName).
     |   mode(Append).
     |   save(basePath){code}
{{}}

{{you will hit the exception quoted above. }}

{{}}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to