You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "Tao Meng (Jira)" <ji...@apache.org> on 2022/02/09 06:57:00 UTC

[jira] [Commented] (HUDI-3347) Updating table schema fails w/ hms mode w/ schema evolution

    [ https://issues.apache.org/jira/browse/HUDI-3347?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17489291#comment-17489291 ] 

Tao Meng commented on HUDI-3347:
--------------------------------

[~shivnarayan] 

This problem seems to be caused by the wrong version of hive jar package

 i try to reproduce the problem  with hive2.3/hive3.1 , but no problem happen.   

i checkout the hive source code, 
org.apache.hadoop.hive.metastore.IMetaStoreClient.alter_table_with_environmentContext
this Interface has not changed  since hive2.0.

could you pls paste your hive/spark version number

> Updating table schema fails w/ hms mode w/ schema evolution
> -----------------------------------------------------------
>
>                 Key: HUDI-3347
>                 URL: https://issues.apache.org/jira/browse/HUDI-3347
>             Project: Apache Hudi
>          Issue Type: Task
>          Components: hive-sync
>            Reporter: sivabalan narayanan
>            Assignee: tao meng
>            Priority: Critical
>             Fix For: 0.11.0
>
>
> When table schema got upgraded with a new batch of write, hms mode sync fails. 
>  
> steps to reproduce using our docker demo set up. 
> I used 0.10.0 to test this out.
> adhoc-1
> {code:java}
> $SPARK_INSTALL/bin/spark-shell --jars $HUDI_SPARK_BUNDLE --master local[2] --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false --deploy-mode client --driver-memory 1G --executor-memory 3G --num-executors 1 --packages org.apache.spark:spark-avro_2.11:2.4.4 {code}
> {code:java}
> import java.sql.Timestamp
> import spark.implicits._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._
> val df1 = Seq(
>         ("row1", 1, "part1" ,1578283932000L ),
>         ("row2", 1, "part1", 1578283942000L)
>       ).toDF("row", "ppath", "preComb","eventTime")
>  df1.write.format("hudi").
>         options(getQuickstartWriteConfigs).
>         option(PRECOMBINE_FIELD_OPT_KEY, "preComb").
>         option(RECORDKEY_FIELD_OPT_KEY, "row").
>         option(PARTITIONPATH_FIELD_OPT_KEY, "ppath").
>         option("hoodie.datasource.write.keygenerator.class","org.apache.hudi.keygen.TimestampBasedKeyGenerator").
>         option("hoodie.deltastreamer.keygen.timebased.timestamp.type","EPOCHMILLISECONDS").
>         option("hoodie.deltastreamer.keygen.timebased.output.dateformat","yyyy/MM/dd").
>         option("hoodie.deltastreamer.keygen.timebased.timezone","GMT+8:00").
>         option("hoodie.datasource.hive_sync.mode","hms").
>         option("hoodie.datasource.hive_sync.database","default").
>         option("hoodie.datasource.hive_sync.table","timestamp_tbl1").
>         option("hoodie.datasource.hive_sync.partition_fields","year,month,day").
>         option("hoodie.datasource.hive_sync.enable","true").
>         option(TABLE_NAME, "timestamp_tbl1").
>         mode(Overwrite).
>         save("/tmp/hudi_timestamp_tbl1")
>  {code}
> // evol schema
> {code:java}
> val df2 = Seq(
>         ("row1", 1, "part1" ,1678283932000L, "abcd" ),
>         ("row2", 1, "part1", 1678283942000L, "defg")
>       ).toDF("row", "ppath", "preComb", "eventTime", "randomStr")
>  df2.write.format("hudi").
>         options(getQuickstartWriteConfigs).
>         option(PRECOMBINE_FIELD_OPT_KEY, "preComb").
>         option(RECORDKEY_FIELD_OPT_KEY, "row").
>         option(PARTITIONPATH_FIELD_OPT_KEY, "ppath").
>         option("hoodie.datasource.write.keygenerator.class","org.apache.hudi.keygen.TimestampBasedKeyGenerator").
>         option("hoodie.deltastreamer.keygen.timebased.timestamp.type","EPOCHMILLISECONDS").
>         option("hoodie.deltastreamer.keygen.timebased.output.dateformat","yyyy/MM/dd").
>         option("hoodie.deltastreamer.keygen.timebased.timezone","GMT+8:00").
>         option("hoodie.datasource.hive_sync.mode","hms").
>         option("hoodie.datasource.hive_sync.database","default").
>         option("hoodie.datasource.hive_sync.table","timestamp_tbl1").
>         option("hoodie.datasource.hive_sync.partition_fields","year,month,day").
>         option("hoodie.datasource.hive_sync.enable","true").
>         option(TABLE_NAME, "timestamp_tbl1").
>         mode(Append).
>         save("/tmp/hudi_timestamp_tbl1")
>  {code}
> stacktrace
> {code:java}
> scala>  df2.write.format("hudi").
>      |         options(getQuickstartWriteConfigs).
>      |         option(PRECOMBINE_FIELD_OPT_KEY, "preComb").
>      |         option(RECORDKEY_FIELD_OPT_KEY, "row").
>      |         option(PARTITIONPATH_FIELD_OPT_KEY, "ppath").
>      |         option("hoodie.datasource.write.keygenerator.class","org.apache.hudi.keygen.TimestampBasedKeyGenerator").
>      |         option("hoodie.deltastreamer.keygen.timebased.timestamp.type","EPOCHMILLISECONDS").
>      |         option("hoodie.deltastreamer.keygen.timebased.output.dateformat","yyyy/MM/dd").
>      |         option("hoodie.deltastreamer.keygen.timebased.timezone","GMT+8:00").
>      |         option("hoodie.datasource.hive_sync.mode","hms").
>      |         option("hoodie.datasource.hive_sync.database","default").
>      |         option("hoodie.datasource.hive_sync.table","timestamp_tbl1").
>      |         option("hoodie.datasource.hive_sync.partition_fields","year,month,day").
>      |         option("hoodie.datasource.hive_sync.enable","true").
>      |         option(TABLE_NAME, "timestamp_tbl1").
>      |         mode(Append).
>      |         save("/tmp/hudi_timestamp_tbl1")
> warning: there was one deprecation warning; re-run with -deprecation for details
> 01:00  WARN: Timeline-server-based markers are not supported for HDFS: base path /tmp/hudi_timestamp_tbl1.  Falling back to direct markers.
> 01:00  WARN: Timeline-server-based markers are not supported for HDFS: base path /tmp/hudi_timestamp_tbl1.  Falling back to direct markers.
> 01:01  WARN: Timeline-server-based markers are not supported for HDFS: base path /tmp/hudi_timestamp_tbl1.  Falling back to direct markers.
> java.lang.NoSuchMethodError: org.apache.hadoop.hive.metastore.IMetaStoreClient.alter_table_with_environmentContext(Ljava/lang/String;Ljava/lang/String;Lorg/apache/hadoop/hive/metastore/api/Table;Lorg/apache/hadoop/hive/metastore/api/EnvironmentContext;)V
>   at org.apache.hudi.hive.ddl.HMSDDLExecutor.updateTableDefinition(HMSDDLExecutor.java:146)
>   at org.apache.hudi.hive.HoodieHiveClient.updateTableDefinition(HoodieHiveClient.java:184)
>   at org.apache.hudi.hive.HiveSyncTool.syncSchema(HiveSyncTool.java:250)
>   at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:182)
>   at org.apache.hudi.hive.HiveSyncTool.doSync(HiveSyncTool.java:131)
>   at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:117)
>   at org.apache.hudi.HoodieSparkSqlWriter$.org$apache$hudi$HoodieSparkSqlWriter$$syncHive(HoodieSparkSqlWriter.scala:537)
>   at org.apache.hudi.HoodieSparkSqlWriter$$anonfun$metaSync$2.apply(HoodieSparkSqlWriter.scala:593)
>   at org.apache.hudi.HoodieSparkSqlWriter$$anonfun$metaSync$2.apply(HoodieSparkSqlWriter.scala:589)
>   at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
>   at org.apache.hudi.HoodieSparkSqlWriter$.metaSync(HoodieSparkSqlWriter.scala:589)
>   at org.apache.hudi.HoodieSparkSqlWriter$.commitAndPerformPostOperations(HoodieSparkSqlWriter.scala:662)
>   at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:282)
>   at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:164)
>   at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
>   at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
>   at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
>   at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:86)
>   at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
>   at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
>   at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
>   at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>   at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
>   at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
>   at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
>   at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
>   at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
>   at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
>   at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
>   at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
>   at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
>   at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
>   at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:285)
>   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
>   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
>   ... 79 elided
> scala>  {code}
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)