[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-28 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-620893610


    BUG status
   
   | bug | status | way |
   | :-| : | :: |
   | upsert long time first time | fixed | upgrate version (0.5.0 to master) |
   | FileStatusExt: Metadata Entry doesn't exist | fixed | using a whole new 
base path |
   |java.io.FileNotFoundException: No such file or directory | fiexed | but 
don't know the root cause |
   | 17 minutes lag between HoodieActiveTimeline and CleanActionExecutor | 
fighting | xxx |



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-27 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-620246831


   ## The lastest spark log (master branch)
   
   ### Simplified origin spark log
   ```
   // Upsert part
   Warning: Ignoring non-spark config property: 
"spark.sql.hive.convertMetastoreParquet=false"
   Warning: Ignoring non-spark config property: 
"spark.serializer=org.apache.spark.serializer.KryoSerializer"
   20/04/26 17:35:58 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object 'jar' in bucket ''. Returning object 
without decryption.
   
   // Start rollback
   20/04/26 17:36:34 INFO HoodieWriteClient: Begin rollback of instant 
20200426170940
   20/04/26 17:36:34 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
from 
   20/04/26 17:36:34 INFO FSUtils: Hadoop Configuration: fs.defaultFS: 
[hdfs://ip-10-0-55-99.ec2.internal:8020]
   
   // Output repeatly 
   20/04/26 17:36:40 INFO FileStatusExt: Metadata Entry doesn't exist
   ...
   20/04/26 18:19:07 INFO FileStatusExt: Metadata Entry doesn't exist
   
   // Finish
   20/04/26 18:19:08 INFO CleanActionExecutor: Total Partitions to clean : 
6052, with policy KEEP_LATEST_COMMITS
   20/04/26 18:19:08 INFO CleanActionExecutor: Using cleanerParallelism: 6000
   20/04/26 18:19:08 INFO SparkContext: Starting job: collect at 
CleanActionExecutor.java:87
   20/04/26 18:19:08 INFO DAGScheduler: Got job 10 (collect at 
CleanActionExecutor.java:87) with 6000 output partitions
   20/04/26 18:19:08 INFO DAGScheduler: Final stage: ResultStage 30 (collect at 
CleanActionExecutor.java:87)
   20/04/26 18:19:08 INFO DAGScheduler: Parents of final stage: List()
   20/04/26 18:19:08 INFO DAGScheduler: Missing parents: List()
   20/04/26 18:19:08 INFO DAGScheduler: Submitting ResultStage 30 
(MapPartitionsRDD[63] at map at CleanActionExecutor.java:86), which has no 
missing parents
   20/04/26 18:19:08 INFO MemoryStore: Block broadcast_20 stored as values in 
memory (estimated size 247.7 KB, free 1008.3 MB)
   20/04/26 18:19:08 INFO MemoryStore: Block broadcast_20_piece0 stored as 
bytes in memory (estimated size 87.1 KB, free 1008.2 MB)
   20/04/26 18:19:08 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory 
on ip-10-0-55-99.ec2.internal:44415 (size: 87.1 KB, free: 1008.8 MB)
   20/04/26 18:19:08 INFO SparkContext: Created broadcast 20 from broadcast at 
DAGScheduler.scala:1201
   20/04/26 18:19:08 INFO DAGScheduler: Submitting 6000 missing tasks from 
ResultStage 30 (MapPartitionsRDD[63] at map at CleanActionExecutor.java:86) 
(first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 
12, 13, 14))
   20/04/26 18:19:08 INFO YarnScheduler: Adding task set 30.0 with 6000 tasks
   20/04/26 18:19:23 INFO YarnScheduler: Removed TaskSet 32.0, whose tasks have 
all completed, from pool 
   20/04/26 18:19:23 INFO DAGScheduler: ResultStage 32 (collect at 
CleanActionExecutor.java:155) finished in 3.444 s
   20/04/26 18:19:23 INFO DAGScheduler: Job 11 finished: collect at 
CleanActionExecutor.java:155, took 8.313284 s
   20/04/26 18:19:23 INFO HoodieActiveTimeline: Loaded instants 
[[20200425201231__clean__COMPLETED], [20200425201231__commit__COMPLETED], 
[==>20200426173625__clean__INFLIGHT], [20200426173625__commit__COMPLETED]]
   20/04/26 18:19:24 INFO HoodieActiveTimeline: Checking for file exists 
?.hoodie/20200426173625.clean.inflight
   20/04/26 18:19:24 INFO CSEMultipartUploadOutputStream: close closed:false 
.hoodie/20200426173625.clean
   20/04/26 18:19:24 INFO DefaultMultipartUploadDispatcher: Completed multipart 
upload of 1 parts 273450 bytes
   20/04/26 18:19:24 INFO CSEMultipartUploadOutputStream: Finished uploading 
.hoodie/20200426173625.clean. Elapsed seconds: 0.
   20/04/26 18:19:24 INFO HoodieActiveTimeline: Create new file for toInstant 
?.hoodie/20200426173625.clean
   20/04/26 18:19:24 INFO CleanActionExecutor: Marked clean started on 
20200426173625 as complete
   20/04/26 18:19:24 INFO AbstractHoodieWriteClient: Committed 20200426173625
   20/04/26 18:19:24 INFO HoodieSparkSqlWriter$: Commit 20200426173625 
successful!
   20/04/26 18:19:28 INFO SparkContext: Invoking stop() from shutdown hook
   20/04/26 18:19:28 INFO SparkUI: Stopped Spark web UI at ip
   20/04/26 18:19:28 INFO YarnClientSchedulerBackend: Interrupting monitor 
thread
   20/04/26 18:19:28 INFO YarnClientSchedulerBackend: Shutting down all 
executors
   20/04/26 18:19:28 INFO YarnSchedulerBackend$YarnDriverEndpoint: Asking each 
executor to shut down
   20/04/26 18:19:28 INFO SchedulerExtensionServices: Stopping 
SchedulerExtensionServices
   (serviceOption=None,
services=List(),
started=false)
   20/04/26 18:19:28 INFO YarnClientSchedulerBackend: Stopped
   20/04/26 18:19:28 INFO MapOutputTrackerMasterEndpoint: 
MapOutputTrackerMasterEndpoint stopped!
   20/04/26 18:19:28 INFO MemoryStore: MemoryStore cleared
   20/04/26 18:19:28 INFO BlockManager: BlockManager stopped
   20/04/26 18:19:28 INFO 

[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-25 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-619449248


   hi @harshi2506, build steps:
   **1. Build Env**
   - JDK8
   - Unix
   
   **2. Commands**
   ```
   git clone https://github.com/apache/incubator-hudi.git
   mvn clean install -DskipTests -DskipITs -Dcheckstyle.skip=true 
-Drat.skip=true
   
   ```
   
   **3. Run env**
   - Spark-2.4.4+
   - avro-1.8.0
   ```
   // run in local env
   export SPARK_HOME=/work/BigData/install/spark/spark-2.4.4-bin-hadoop2.7
   ${SPARK_HOME}/bin/spark-shell \
 --driver-memory 6G \
 --packages org.apache.spark:spark-avro_2.11:2.4.4 \
 --jars `ls 
packaging/hudi-spark-bundle/target/hudi-spark-bundle_*.*-*.*.*-SNAPSHOT.jar` \
 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'
   
   // run in yarn env
   export SPARK_HOME=/BigData/install/spark-2.4.4-bin-hadoop2.7
   ${SPARK_HOME}/bin/spark-shell \
 --master yarn \
 --driver-memory 6G \
 --executor-memory 6G \
 --num-executors 5 \
 --executor-cores 5 \
 --queue root.default \
 --packages org.apache.spark:spark-avro_2.11:2.4.4 \
 --jars `ls 
packaging/hudi-spark-bundle/target/hudi-spark-bundle_*.*-*.*.*-SNAPSHOT.jar` \
 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'
   
   // scripts
   import org.apache.spark.sql.functions._
   
   val tableName = "hudi_mor_table"
   val basePath = "file:///tmp/hudi_mor_tablen"
   // val basePath = "hdfs:///hudi/test"
   
   val hudiOptions = Map[String,String](
 "hoodie.upsert.shuffle.parallelism" -> "10",
 "hoodie.datasource.write.recordkey.field" -> "key",
 "hoodie.datasource.write.partitionpath.field" -> "dt", 
 "hoodie.table.name" -> tableName,
 "hoodie.datasource.write.precombine.field" -> "timestamp"
   )
   
   val inputDF = spark.range(1, 7).
  withColumn("key", $"id").
  withColumn("data", lit("data")).
  withColumn("timestamp", unix_timestamp()).
  withColumn("dtstamp", unix_timestamp() + ($"id" * 24 * 3600)).
  withColumn("dt", from_unixtime($"dtstamp", "/MM/dd"))
   
   inputDF.write.format("org.apache.hudi").
 options(hudiOptions).
 mode("Overwrite").
 save(basePath)
   
   spark.read.format("org.apache.hudi").load(basePath + "/*/*/*").show();
   ```



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-24 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-619308174


   > @harshi2506 I am suspecting this may be due to a recent bug we fixed on 
master (still not 100%). Are you open to building hudi off master branch and 
giving that a shot? I am suspecting #1394
   
   IMO, this solution will help you, please try to building hudi master branch. 
@harshi2506 



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-24 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-619306738


   ## Spark log analysis brainstorming 
   
   hi @vinothchandar, preliminary analysis about this issue
   (based on `release-0.5.0` branch 
https://github.com/apache/incubator-hudi/tree/release-0.5.0)
   
   ### Simplified origin spark log
   
   ```
   // Upsert part
   Warning: Ignoring non-spark config property: 
"spark.sql.hive.convertMetastoreParquet=false"
   Warning: Ignoring non-spark config property: 
"spark.serializer=org.apache.spark.serializer.KryoSerializer"
   20/04/22 20:12:14 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object '' in bucket ''. Returning 
object without decryption.
   20/04/22 20:12:14 WARN HiveConf: HiveConf of name hive.server2.thrift.url 
does not exist
   20/04/22 20:12:14 INFO SparkContext: Running Spark version 2.4.4
   20/04/22 20:12:14 INFO SparkContext: Submitted application: HUDI incremental 
data loading
   20/04/22 20:12:15 INFO SecurityManager: Changing view acls to: hadoop
   20/04/22 20:12:15 INFO SecurityManager: Changing modify acls to: hadoop
   20/04/22 20:12:15 INFO SecurityManager: Changing view acls groups to: 
   20/04/22 20:15:31 INFO S3NativeFileSystem: Opening 
'.hoodie/hoodie.properties' for reading
   20/04/22 20:15:31 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object '/.hoodie/hoodie.properties' in bucket 
''. Returning object without decryption.
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Finished Loading Table of type 
COPY_ON_WRITE from 
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Loading Active commit timeline 
for 
   20/04/22 20:15:31 INFO HoodieActiveTimeline: Loaded instants 
java.util.stream.ReferencePipeline$Head@678852b5
   20/04/22 20:24:33 INFO HoodieCopyOnWriteTable: Partitions to clean up : 
[2001/05/02, 2001/05/07, 2001/05/09, 2001/05/10, 2001/05/17, 2001/05/18, 
2001/05/21, 2001/06/01, 2001/06/04, 2001/06/08, 2001/06/20, 2001/06/21, 
2001/07/17, 2001/07/23, 2001/07/25, 2001/07/30, 2001/08/02, 2001/08/03, 
2001/08/07, 2001/08/08, 2001/08/09, 2001/08/14, 2001/08/23, 2001/09/05, 
2001/09/06, 2001/09/07, 2001/09/13, 2001/09/14, 2001/10/02, 2001/10/03, 
2001/10/04, 2001/10/09, 2001/11/01, 2001/11/09, 2001/11/14, 2001/11/15, 
2001/11/16, 2001/11/19, 2001/11/20, 2001/11/21,
   20/04/22 20:24:36 INFO HoodieWriteClient: Marked clean started on 
20200422201250 as complete
   20/04/22 20:24:36 INFO HoodieWriteClient: Committed 20200422201250
   20/04/22 20:24:36 INFO HoodieSparkSqlWriter$: Commit 20200422201250 
successful!
   
   
   // CreateRelation(sqlContext, parameters, df.schema)
   20/04/22 20:24:36 INFO DefaultSource: Constructing hoodie (as parquet) data 
source with options ...
   
   20/04/22 20:24:38 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
from 
   20/04/22 20:24:38 INFO FSUtils: Hadoop Configuration: fs.defaultFS:...], 
FileSystem: [com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem@3c0cfaad]
   ...
   20/04/22 20:24:38 INFO S3NativeFileSystem: Opening 
'2001/05/10/.hoodie_partition_metadata' for reading
   20/04/22 20:24:38 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object 
'/2001/05/10/.hoodie_partition_metadata' in bucket 
''. Returning object without decryption.
   
   20/04/22 20:24:38 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
from 
   20/04/22 20:24:38 INFO FSUtils: Hadoop Configuration: fs.defaultFS:...], 
FileSystem: [com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem@3c0cfaad]
   ...
   20/04/22 20:24:39 INFO S3NativeFileSystem: Opening 
'2001/05/17/.hoodie_partition_metadata' for reading
   20/04/22 20:24:39 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object 
'/2001/05/17/.hoodie_partition_metadata' in bucket 
''. Returning object without decryption.
   
   20/04/22 20:24:39 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
from 
   20/04/22 20:24:39 INFO FSUtils: Hadoop Configuration: fs.defaultFS:...], 
FileSystem: [com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem@3c0cfaad]
   ...
   20/04/22 20:24:39 INFO S3NativeFileSystem: Opening 
'2001/05/18/.hoodie_partition_metadata' for reading
   20/04/22 20:24:39 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object 
'/2001/05/18/.hoodie_partition_metadata' in bucket 
''. Returning object without decryption.
   
   // Finish part
   20/04/22 21:06:25 INFO SparkContext: Successfully stopped SparkContext
   20/04/22 21:06:25 INFO ShutdownHookManager: Shutdown hook called
   20/04/22 21:06:25 INFO ShutdownHookManager: Deleting directory 
/mnt/tmp/spark-c24e90ad-7be5-4a04-b4f1-2575cf68bd5a
   20/04/22 21:06:25 INFO ShutdownHookManager: Deleting directory 
/mnt/tmp/spark-c18b2f4f-e525-43de-b896-75244fe63591
   ```
   
    Analysis
   
   
![image](https://user-images.githubusercontent.com/20113411/80269021-c7225880-86de-11ea-84fe-464d08fc4e9b.png)
   
   
   ### Analysis about 

[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-24 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-619278766


   @harshi2506, I tried to reproduce your steps, maybe got the answer
   
   - Generate about 7000 partitions
   - Upsert 300 partitions
   
   From following demo code
   
   - `sc.setLogLevel("INFO")`, upsert cost 15min
   - `sc.setLogLevel("WARN")`, upsert cost 3min
   
   ```
   export HADOOP_HOME=/BigData/install/hadoop/hadoop-2.7.7-streamjar
   export HADOOP_MAPRED_HOME=$HADOOP_HOME
   export HADOOP_COMMON_HOME=$HADOOP_HOME
   export HADOOP_HDFS_HOME=$HADOOP_HOME
   export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop
   export YARN_HOME=$HADOOP_HOME
   export YARN_CONF_DIR=$HADOOP_HOME/etc/hadoop
   export HADOOP_USER_NAME=dcadmin
   
   export SPARK_HOME=/BigData/install/spark-2.4.4-bin-hadoop2.7
   ${SPARK_HOME}/bin/spark-shell \
 --master yarn \
 --driver-memory 6G \
 --executor-memory 6G \
 --num-executors 5 \
 --executor-cores 5 \
 --queue root.default \
 --packages 
org.apache.hudi:hudi-spark-bundle:0.5.0-incubating,org.apache.spark:spark-avro_2.11:2.4.4
 \
 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'
   
   import org.apache.spark.sql.functions._
   
   val tableName = "hudi_mor_table"
   val basePath = "viewfs://dcfs/ns-log/wxdprtc/flink/hudi"
   
   val hudiOptions = Map[String,String](
 "hoodie.upsert.shuffle.parallelism" -> "10",
 "hoodie.datasource.write.recordkey.field" -> "key",
 "hoodie.datasource.write.partitionpath.field" -> "dt", 
 "hoodie.table.name" -> tableName,
 "hoodie.datasource.write.precombine.field" -> "timestamp"
   )
   
   val inputDF = spark.range(1, 7000).
  withColumn("key", $"id").
  withColumn("data", lit("data")).
  withColumn("timestamp", unix_timestamp()).
  withColumn("dtstamp", unix_timestamp() + ($"id" * 24 * 3600)).
  withColumn("dt", from_unixtime($"dtstamp", "/MM/dd"))
   
   inputDF.write.format("org.apache.hudi").
 options(hudiOptions).
 mode("Overwrite").
 save(basePath)
   
   spark.read.format("org.apache.hudi").load(basePath + "/*/*/*").count();
   
   
   // cost long time, about 15min
   sc.setLogLevel("INFO")
   
   val inputDF = spark.range(1, 300).
  withColumn("key", $"id").
  withColumn("data", lit("data")).
  withColumn("timestamp", unix_timestamp()).
  withColumn("dtstamp", unix_timestamp() + ($"id" * 24 * 3600)).
  withColumn("dt", from_unixtime($"dtstamp", "/MM/dd"))
   
   inputDF.write.format("org.apache.hudi").
 options(hudiOptions).
 mode("Append").
 save(basePath)
   
   
   // cost less time, about 3min
   sc.setLogLevel("WARN")
   
   val inputDF = spark.range(1, 300).
  withColumn("key", $"id").
  withColumn("data", lit("data")).
  withColumn("timestamp", unix_timestamp()).
  withColumn("dtstamp", unix_timestamp() + ($"id" * 24 * 3600)).
  withColumn("dt", from_unixtime($"dtstamp", "/MM/dd"))
   
   inputDF.write.format("org.apache.hudi").
 options(hudiOptions).
 mode("Append").
 save(basePath)
   ```
   
   



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-23 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-618683897


   hi @harshi2506, need more spark info log, you can put the logfile to google 
drive, 
   e.g https://drive.google.com/file/d/1zzyaySDJqPgAdTSLnKwOG667QGvZhd03



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-23 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-618657572


   Upsert hoodie log, cost about 30min
   ```
   Started at 20/04/22 20:12:14 
   
   
   20/04/22 20:15:30 INFO HoodieTableMetaClient: Finished Loading Table of type 
COPY_ON_WRITE from 
   20/04/22 20:15:30 INFO HoodieTableMetaClient: Loading Active commit timeline 
for 
   20/04/22 20:15:30 INFO HoodieActiveTimeline: Loaded instants 
java.util.stream.ReferencePipeline$Head@4af81941
   20/04/22 20:15:30 INFO HoodieCommitArchiveLog: No Instants to archive
   20/04/22 20:15:30 INFO HoodieWriteClient: Auto cleaning is enabled. Running 
cleaner now
   20/04/22 20:15:30 INFO HoodieWriteClient: Cleaner started
   20/04/22 20:15:30 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
from 
   20/04/22 20:15:30 INFO FSUtils: Hadoop Configuration: fs.defaultFS:hdfs, 
Config:[Configuration: core-default.xml, core-site.xml, mapred-default.xml, 
mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, 
hdfs-site.xml, emrfs-site.xml, __spark_hadoop_conf__.xml, 
file:/etc/spark/conf.dist/hive-site.xml], FileSystem: 
[com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem@3c0cfaad]
   20/04/22 20:15:31 INFO HoodieTableConfig: Loading dataset properties from 
.hoodie/hoodie.properties
   20/04/22 20:15:31 INFO S3NativeFileSystem: Opening 
.hoodie/hoodie.properties' for reading
   20/04/22 20:15:31 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object '.hoodie/hoodie.properties' in bucket 
'delta-data-devo'. Returning object without decryption.
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Finished Loading Table of type 
COPY_ON_WRITE from 
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Loading Active commit timeline 
for 
   20/04/22 20:15:31 INFO HoodieActiveTimeline: Loaded instants 
java.util.stream.ReferencePipeline$Head@72dd302
   20/04/22 20:15:31 INFO FileSystemViewManager: Creating View Manager with 
storage type :MEMORY
   20/04/22 20:15:31 INFO FileSystemViewManager: Creating in-memory based Table 
View
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
from 
   20/04/22 20:15:31 INFO FSUtils: Hadoop Configuration: fs.defaultFS: hdfs, 
Config:[Configuration: core-default.xml, core-site.xml, mapred-default.xml, 
mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, 
hdfs-site.xml, emrfs-site.xml, __spark_hadoop_conf__.xml, 
file:/etc/spark/conf.dist/hive-site.xml], FileSystem: 
[com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem@3c0cfaad]
   20/04/22 20:15:31 INFO HoodieTableConfig: Loading dataset properties from 
.hoodie/hoodie.properties
   20/04/22 20:15:31 INFO S3NativeFileSystem: Opening 
'.hoodie/hoodie.properties' for reading
   20/04/22 20:15:31 WARN S3CryptoModuleAE: Unable to detect encryption 
information for object '.hoodie/hoodie.properties' in bucket 
'delta-data-devo'. Returning object without decryption.
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Finished Loading Table of type 
COPY_ON_WRITE from 
   20/04/22 20:15:31 INFO HoodieTableMetaClient: Loading Active commit timeline 
for 
   20/04/22 20:15:31 INFO HoodieActiveTimeline: Loaded instants 
java.util.stream.ReferencePipeline$Head@678852b5
   20/04/22 20:24:33 INFO HoodieCopyOnWriteTable: Partitions to clean up : 
[2001/05/02, 2001/05/07, 2001/05/09, 2001/05/10, 2001/05/17, 2001/05/18, 
2001/05/21, 2001/06/01, 2001/06/04, 2001/06/08, 2001/06/20, 2001/06/21, 
2001/07/17, 2001/07/23, 2001/07/25, 2001/07/30, 2001/08/02, 2001/08/03, 
2001/08/07, 2001/08/08, 2001/08/09, 2001/08/14, 2001/08/23, 2001/09/05, 
2001/09/06, 2001/09/07, 2001/09/13, 2001/09/14, 2001/10/02, 2001/10/03, 
2001/10/04, 2001/10/09, 2001/11/01, 2001/11/09, 2001/11/14, 2001/11/15, 
2001/11/16, 2001/11/19, 2001/11/20, 2001/11/21, 2001/11/27, 2001/11/28, 
2001/11/29, 2001/11/30, 2001/12/03, 2001/12/07, 2001/12/10, 2001/12/11, 
2001/12/12, 2001/12/13, 2001/12/17, 2001/12/20, 2001/12/21, 2001/12/25, 
2001/12/26, 2001/12/27, 2001/12/28, 2001/12/29, 2001/12/31, 2002/01/02, 
2002/01/03, 2002/01/07, 2002/01/08, 2002/01/09, 2002/01/11, 2002/01/13, 
2002/01/14, 2002/01/15, 2002/01/16, 2002/01/17, 2002/01/18, 2002/01/21, 
2002/01/22, 2002/01/23, 2002/01/25, 2002/01/28, 2002/01/29, 2002/01/30, 
2002/02/03, 2002/02/05, 2002/02/06, 2002/02/07, 2002/02/11, 2002/02/12, 
2002/02/14, 2002/02/15, 2002/02/18, 2002/02/19, 2002/02/20, 2002/02/21, 
2002/02/22, 2002/02/26, 2002/03/02, 2002/03/04, 2002/03/06, 2002/03/10, 
2002/03/15, 2002/03/17, 2002/03/19, 2002/03/20, 2002/03/21, 2002/03/22, 
2002/03/25, 2002/03/26, 2002/03/27, 2002/03/28, 2002/03/30, 2002/04/02, 
2002/04/03, 2002/04/04, 2002/04/05, 2002/04/07, 2002/04/09, 2002/04/10, 
2002/04/11, 2002/04/14, 2002/04/16, 2002/04/17, 2002/04/22, 2002/04/23, 
2002/04/25, 2002/04/30, 2002/05/01, 2002/05/02, 2002/05/06, 2002/05/08, 
2002/05/09, 2002/05/12, 2002/05/13, 2002/05/14, 2002/05/17, 2002/05/19, 
2002/05/20, 2002/05/21, 2002/05/22, 

[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-22 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-617976874


   hi @harshi2506, based on the above analysis, please try to increate the 
upsert parallelism and spark executor instances, for example
   ```
   export SPARK_HOME=/work/BigData/install/spark/spark-2.4.4-bin-hadoop2.7
   ${SPARK_HOME}/bin/spark-shell \
 --master yarn \
 --driver-memory 6G \
 --num-executors 10 \
 --executor-cores 5 \
 --packages 
org.apache.hudi:hudi-spark-bundle_2.11:0.5.1-incubating,org.apache.spark:spark-avro_2.11:2.4.4
 \
 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'
   
   import org.apache.spark.sql.functions._
   
   val tableName = "hudi_mor_table"
   val basePath = "file:///tmp/hudi_mor_tablen"
   
   val hudiOptions = Map[String,String](
 "hoodie.upsert.shuffle.parallelism" -> "200",
 "hoodie.datasource.write.recordkey.field" -> "id",
 "hoodie.datasource.write.partitionpath.field" -> "key", 
 "hoodie.table.name" -> tableName,
 "hoodie.datasource.write.precombine.field" -> "timestamp",
 "hoodie.memory.merge.max.size" -> "200485760"
   )
   
   val inputDF = spark.range(1, 300).
  withColumn("key", $"id").
  withColumn("data", lit("data")).
  withColumn("timestamp", current_timestamp()).
  withColumn("dt", date_format($"timestamp", "-MM-dd"))
   
   inputDF.write.format("org.apache.hudi").
 options(hudiOptions).
 mode("Append").
 save(basePath)
   
   spark.read.format("org.apache.hudi").load(basePath + "/*/*").show();
   ```



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-22 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-617965381


   From the detailed commit metadata and above spark ui, we know
   
   1. write about 700 million records at first commit.
   2. upsert 2633 records and touched 255 partitions.
   
   **Key point**: small input upsert to large old data file slide, and touched 
many partitions.
   
   This is a different case, so the `hoodie.memory.merge.max.size` option will 
not work, 
   So I think it takes a lot of time to read the old file, WDYT? @vinothchandar 
   
   
![image](https://user-images.githubusercontent.com/20113411/80020370-7f97a300-850b-11ea-9489-8e7002e98cda.png)
   



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [incubator-hudi] lamber-ken commented on issue #1552: Time taken for upserting hudi table is increasing with increase in number of partitions

2020-04-22 Thread GitBox


lamber-ken commented on issue #1552:
URL: https://github.com/apache/incubator-hudi/issues/1552#issuecomment-617895350


   Hi @harshi2506, please share the Spark stage UI, thanks



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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org