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

Vinoth Chandar edited comment on HUDI-254 at 9/17/19 3:49 AM:
--------------------------------------------------------------

h3. Spark 2.3.1 on master 

 

Once I copy the hudi-spark-bundle (had to shade com.databricks:spark-avro* for 
now) to jars, I can do *a, b,c, d* 
{code:java}
root@adhoc-2: cp $HUDI_SPARK_BUNDLE /opt/spark/jars/

root@adhoc-2:/opt# $SPARK_INSTALL/bin/spark-shell --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
19/09/17 03:46:43 WARN util.NativeCodeLoader: Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
setLogLevel(newLevel).
Spark context Web UI available at http://adhoc-2:4040
Spark context available as 'sc' (master = local[2], app id = 
local-1568692009107).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 2.3.1
      /_/Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_212)
Type in expressions to have them evaluated.
Type :help for more information.scala> :paste
// Entering paste mode (ctrl-D to finish)val jsonDF = 
spark.read.json("file:////var/hoodie/ws/docker/demo/data/batch_1.json")import 
org.apache.hudi.DataSourceReadOptions;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.spark.sql.SaveMode;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.HoodieDataSourceHelpers;
import org.apache.hadoop.fs.FileSystem;jsonDF.write.format("org.apache.hudi").
    option("hoodie.insert.shuffle.parallelism", "2").
    option("hoodie.upsert.shuffle.parallelism","2").
    option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, 
DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL).
    option(DataSourceWriteOptions.OPERATION_OPT_KEY, 
DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL).
    option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "key").
    option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "date").
    option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts").
    option(HoodieWriteConfig.TABLE_NAME, "stock_ticks_derived_mor").
    option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY, 
"stock_ticks_derived_mor").
    option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY, "default").
    option(DataSourceWriteOptions.HIVE_URL_OPT_KEY, 
"jdbc:hive2://hiveserver:10000").
    option(DataSourceWriteOptions.HIVE_USER_OPT_KEY, "hive").
    option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive").
    option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true").
    option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr").
    mode(SaveMode.Append).
    
save("file:///tmp/stock_ticks_derived_mor");spark.read.format("org.apache.hudi").load("file:///tmp/stock_ticks_derived_mor/*/*/*/*.parquet").showspark.sql("select
 symbol, max(ts) from stock_ticks_derived_mor group by symbol HAVING symbol = 
'GOOG'").show(100, false)// Exiting paste mode, now interpreting.19/09/17 
03:47:13 WARN compact.HoodieRealtimeTableCompactor: After filtering, Nothing to 
compact for file:///tmp/stock_ticks_derived_mor
+-------------------+--------------------+------------------+----------------------+--------------------+------+----------+---+-------+------------------+------+-----+-------+------+-------------------+------+----+
|_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|
   _hoodie_file_name| close|      date|day|   high|               key|   
low|month|   open|symbol|                 ts|volume|year|
+-------------------+--------------------+------------------+----------------------+--------------------+------+----------+---+-------+------------------+------+-----+-------+------+-------------------+------+----+
|     20190917034711|  20190917034711_0_1|NIHD_2018-08-31 10|            
2018/08/31|5d4f52d9-c16f-4a2...|  5.67|2018/08/31| 31|   5.67|NIHD_2018-08-31 
10|  5.67|   08|   5.67|  NIHD|2018-08-31 10:29:00|  2125|2018|
|     20190917034711|  20190917034711_0_2|CDMO_2018-08-31 09|            
....  
2018/08/31|5d4f52d9-c16f-4a2...|  14.5|2018/08/31| 31|14.5999|EGAN_2018-08-31 
09|  14.5|   08|14.5999|  EGAN|2018-08-31 09:57:00|  2489|2018|
+-------------------+--------------------+------------------+----------------------+--------------------+------+----------+---+-------+------------------+------+-----+-------+------+-------------------+------+----+
only showing top 20 rowsSLF4J: Failed to load class 
"org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
details.
19/09/17 03:47:18 WARN hadoop.ParquetRecordReader: Can not initialize counter 
due to context is not a instance of TaskInputOutputContext, but is 
org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+------+-------------------+
|symbol|max(ts)            |
+------+-------------------+
|GOOG  |2018-08-31 10:29:00|
+------+-------------------+jsonDF: org.apache.spark.sql.DataFrame = [close: 
double, date: string ... 10 more fields]
import org.apache.hudi.DataSourceReadOptions
import org.apache.hudi.DataSourceWriteOptions
import org.apache.spark.sql.SaveMode
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.HoodieDataSourceHelpers
import org.apache.hadoop.fs.FileSystemscala>{code}


was (Author: vc):
h3. Spark 2.3.3 on master 

 

Once I copy the hudi-spark-bundle (had to shade com.databricks:spark-avro* for 
now) to jars, I can do *a + b* 
{code:java}
root@adhoc-2:/var/hoodie/ws/docker# $SPARK_INSTALL/bin/spark-shell --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
19/09/17 00:48:48 WARN util.NativeCodeLoader: Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
setLogLevel(newLevel).
Spark context Web UI available at http://adhoc-2:4040
Spark context available as 'sc' (master = local[2], app id = 
local-1568681334864).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 2.3.1
      /_/Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_212)
Type in expressions to have them evaluated.
Type :help for more information.scala> :paste
// Entering paste mode (ctrl-D to finish)
val jsonDF = 
spark.read.json("file:////var/hoodie/ws/docker/demo/data/batch_1.json")
import org.apache.hudi.DataSourceReadOptions;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.spark.sql.SaveMode;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.HoodieDataSourceHelpers;
import org.apache.hadoop.fs.FileSystem;jsonDF.write.format("org.apache.hudi").
    option("hoodie.insert.shuffle.parallelism", "2").
    option("hoodie.upsert.shuffle.parallelism","2").
    option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, 
DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL).
    option(DataSourceWriteOptions.OPERATION_OPT_KEY, 
DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL).
    option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "key").
    option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "date").
    option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts").
    option(HoodieWriteConfig.TABLE_NAME, "stock_ticks_derived_mor").
    mode(SaveMode.Append).
    
save("file:///tmp/stock_ticks_derived_mor");spark.read.format("org.apache.hudi").load("file:///tmp/stock_ticks_derived_mor/*/*/*/*.parquet").show//
 Exiting paste mode, now 
interpreting.+-------------------+--------------------+------------------+----------------------+--------------------+------+----------+---+-------+------------------+------+-----+-------+------+-------------------+------+----+
|_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|
   _hoodie_file_name| close|      date|day|   high|               key|   
low|month|   open|symbol|                 ts|volume|year|
+-------------------+--------------------+------------------+----------------------+--------------------+------+----------+---+-------+------------------+------+-----+-------+------+-------------------+------+----+
|     20190917004922|  20190917004922_0_1|NIHD_2018-08-31 10|            
2018/08/31|0488121d-4ff5-4fb...|  5.67|2018/08/31| 31|   5.67|NIHD_2018-08-31 
10|  5.67|   08|   5.67|  NIHD|2018-08-31 10:29:00|  2125|2018|
  ...
|     20190917004922| 20190917004922_0_19|STAA_2018-08-31 10|            
2018/08/31|0488121d-4ff5-4fb...|  47.5|2018/08/31| 31|   47.5|STAA_2018-08-31 
10|  47.5|   08|   47.5|  STAA|2018-08-31 10:28:00|   800|2018|
|     20190917004922| 20190917004922_0_20|EGAN_2018-08-31 09|            
2018/08/31|0488121d-4ff5-4fb...|  14.5|2018/08/31| 31|14.5999|EGAN_2018-08-31 
09|  14.5|   08|14.5999|  EGAN|2018-08-31 09:57:00|  2489|2018|
+-------------------+--------------------+------------------+----------------------+--------------------+------+----------+---+-------+------------------+------+-----+-------+------+-------------------+------+----+
only showing top 20 rows {code}

> Provide mechanism for installing hudi-spark-bundle onto an existing spark 
> installation
> --------------------------------------------------------------------------------------
>
>                 Key: HUDI-254
>                 URL: https://issues.apache.org/jira/browse/HUDI-254
>             Project: Apache Hudi (incubating)
>          Issue Type: Improvement
>          Components: Spark datasource, SparkSQL Support
>            Reporter: Vinoth Chandar
>            Assignee: Vinoth Chandar
>            Priority: Major
>
> A lot of discussions around this kicked off from 
> [https://github.com/apache/incubator-hudi/issues/869] 
> Breaking down into phases, when we drop the hudi-spark-bundle*.jar onto the 
> `jars` folder 
>  
> a) Writing data via Hudi datasource should work 
> b) Spark datasource reads should work
>  
> c)  a + Hive Sync should work
> d) SparkSQL on Hive synced table works 
>  
> Start with Spark 2.3 (current demo setup) and then proceed to 2.4 and iron 
> out issues.
>  



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to