nsivabalan edited a comment on issue #4700:
URL: https://github.com/apache/hudi/issues/4700#issuecomment-1025712431


   Did you try via jdbc. I could able to get it working. with hms, I am running 
into some other issue though. 
   ```
   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> 
   
   ```
   
   
   Steps that worked w/ jdbc:
   
   1. set up docker demo script.
   2. adhoc-1 : launch spark shell
   
   $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
   
   ```
   
   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","jdbc").
           
option("hoodie.datasource.hive_sync.jdbcurl","jdbc:hive2://hiveserver:10000").
           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")
   
   
   // schema evolution
   
   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","jdbc").
           
option("hoodie.datasource.hive_sync.jdbcurl","jdbc:hive2://hiveserver:10000").
           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")
   
   ```
   
   // check beeline
   adhoc-2
   
   ```
   beeline -u jdbc:hive2://hiveserver:10000   --hiveconf 
hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat   --hiveconf 
hive.stats.autogather=false
   
   desc timestamp_tbl1;
   
+--------------------------+-----------------------+-----------------------+--+
   |         col_name         |       data_type       |        comment        |
   
+--------------------------+-----------------------+-----------------------+--+
   | _hoodie_commit_time      | string                |                       |
   | _hoodie_commit_seqno     | string                |                       |
   | _hoodie_record_key       | string                |                       |
   | _hoodie_partition_path   | string                |                       |
   | _hoodie_file_name        | string                |                       |
   | row                      | string                |                       |
   | ppath                    | int                   |                       |
   | precomb                  | string                |                       |
   | eventtime                | bigint                |                       |
   | randomstr                | string                |                       |
   | year                     | string                |                       |
   | month                    | string                |                       |
   | day                      | string                |                       |
   |                          | NULL                  | NULL                  |
   | # Partition Information  | NULL                  | NULL                  |
   | # col_name               | data_type             | comment               |
   |                          | NULL                  | NULL                  |
   | year                     | string                |                       |
   | month                    | string                |                       |
   | day                      | string                |                       |
   
+--------------------------+-----------------------+-----------------------+--+
   
   ```
   
   randomStr is the new column added.
   
   
   
   
   
   


-- 
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