You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/10 05:52:58 UTC

[GitHub] [hudi] maheshguptags opened a new issue, #6903: Offline compaction scheduling not working

maheshguptags opened a new issue, #6903:
URL: https://github.com/apache/hudi/issues/6903

   Hi Team,
   
   **I am trying to perform offline compaction using hudi MOR table using spark.**
   
   I am trying to perform offline compaction using Hudi MOR table using spark. for that I have setup in-line schedule using spark code and for execution I am using the HoodieCompactor class.
   
   
   **To Reproduce**
   
   Steps to reproduce the behaviour:
   
   1.Scheduling configuration
   
   ``` hudi_options_write = {
       'hoodie.datasource.write.table.type' : 'MERGE_ON_READ',
       'hoodie.datasource.write.recordkey.field': 'a,b,c',
       'hoodie.table.name': tableName,
       'hoodie.datasource.write.hive_style_partitioning':'false',
       'hoodie.archivelog.folder':'archived',
       'hoodie.datasource.write.operation': 'upsert',
       'hoodie.datasource.write.partitionpath.field': 'a',
       'hoodie.datasource.write.keygenerator.class': 'org.apache.hudi.keygen.ComplexKeyGenerator', ## to allow the multiple key
       'hoodie.datasource.write.partitionpath.urlencode':'false',
       'hoodie.upsert.shuffle.parallelism': 2,
       'hoodie.timeline.layout.version':1,
       'hoodie.datasource.write.precombine.field': 'b' ,
       'hoodie.compact.inline': 'false',
       'hoodie.datasource.compaction.async.enable':'false',
       'hoodie.compact.schedule.inline': 'true',
       'hoodie.compact.inline.max.delta.commits':5,
       'hoodie.table.timeline.timezone':'utc'
   }
   ```
   
   2. execution using below class 
    ``` spark-submit  --class org.apache.hudi.utilities.HoodieCompactor --jars /usr/lib/hudi/hudi-spark3-bundle_2.12-0.10.1-amzn-0.jar /usr/lib/hudi/hudi-utilities-bundle_2.12-0.10.1-amzn-0.jar --base-path "s3://test-spark-hudi/test_campaign_event_offline_compact_v1/" --table-name "customer_event_offline_v1" --schema-file "s3://test-spark-hudi/schema/offline_compact.avsc" --schedule  --strategy "org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy" --instant-time "20221007120816651"  --spark-memory 1g --parallelism 2 ```
   
   **Expected behavior**
   the outcome of scheduling code is that the spark code must generate the `compact.requested` file after every five delta log commit as per the default behaviour. but it is not generating. 
   
   Secondly when I try to run the scheduling from **`hudi-cli>`** using `compaction schedule` so its behaviour is random(sometime works and sometime doesn't) not sure why? I have also attached the stack trace for same.    
   
   **Environment Description**
   
   * EMR Version  : emr-6.6.0
   * Hudi version : 10.1 & 11(tried on both)
   
   * Spark version : Spark  3.2.0-amzn-0
   
   * Hive version : Hive 3.1.2
   
   * Hadoop version : Hadoop Amazon 3.2.1
   
   * Storage (HDFS/S3/GCS..) : S3
   
   * Running on Docker? (yes/no) : NO 
   
   
   **Additional context**
   
   **Stacktrace**
   
   ```compaction schedule 
   22/10/10 05:39:45 INFO SparkContext: Running Spark version 3.2.0-amzn-0
   22/10/10 05:39:45 INFO ResourceUtils: ==============================================================
   22/10/10 05:39:45 INFO ResourceUtils: No custom resources configured for spark.driver.
   22/10/10 05:39:45 INFO ResourceUtils: ==============================================================
   22/10/10 05:39:45 INFO SparkContext: Submitted application: hoodie-cli-COMPACT_SCHEDULE
   22/10/10 05:39:45 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 4, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 0, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0)
   22/10/10 05:39:45 INFO ResourceProfile: Limiting resource is cpus at 4 tasks per executor
   22/10/10 05:39:45 INFO ResourceProfileManager: Added ResourceProfile id: 0
   22/10/10 05:39:45 INFO SecurityManager: Changing view acls to: hadoop
   22/10/10 05:39:45 INFO SecurityManager: Changing modify acls to: hadoop
   22/10/10 05:39:45 INFO SecurityManager: Changing view acls groups to: 
   22/10/10 05:39:45 INFO SecurityManager: Changing modify acls groups to: 
   22/10/10 05:39:45 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(hadoop); groups with view permissions: Set(); users  with modify permissions: Set(hadoop); groups with modify permissions: Set()
   22/10/10 05:39:45 INFO deprecation: mapred.output.compression.codec is deprecated. Instead, use mapreduce.output.fileoutputformat.compress.codec
   22/10/10 05:39:45 INFO deprecation: mapred.output.compression.type is deprecated. Instead, use mapreduce.output.fileoutputformat.compress.type
   22/10/10 05:39:45 INFO deprecation: mapred.output.compress is deprecated. Instead, use mapreduce.output.fileoutputformat.compress
   22/10/10 05:39:45 INFO Utils: Successfully started service 'sparkDriver' on port 37043.
   22/10/10 05:39:45 INFO SparkEnv: Registering MapOutputTracker
   22/10/10 05:39:45 INFO SparkEnv: Registering BlockManagerMaster
   22/10/10 05:39:45 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information
   22/10/10 05:39:45 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up
   22/10/10 05:39:45 INFO SparkEnv: Registering BlockManagerMasterHeartbeat
   22/10/10 05:39:45 INFO DiskBlockManager: Created local directory at /mnt/tmp/blockmgr-a788d127-7fc5-4af7-99c1-3867375f3887
   22/10/10 05:39:45 INFO MemoryStore: MemoryStore started with capacity 912.3 MiB
   22/10/10 05:39:45 INFO SparkEnv: Registering OutputCommitCoordinator
   22/10/10 05:39:45 INFO SubResultCacheManager: Sub-result caches are disabled.
   22/10/10 05:39:45 INFO log: Logging initialized @2490ms to org.sparkproject.jetty.util.log.Slf4jLog
   22/10/10 05:39:45 INFO Server: jetty-9.4.43.v20210629; built: 2021-06-30T11:07:22.254Z; git: 526006ecfa3af7f1a27ef3a288e2bef7ea9dd7e8; jvm 1.8.0_342-b07
   22/10/10 05:39:45 INFO Server: Started @2596ms
   22/10/10 05:39:46 INFO AbstractConnector: Started ServerConnector@ecfbe91{HTTP/1.1, (http/1.1)}{0.0.0.0:4040}
   22/10/10 05:39:46 INFO Utils: Successfully started service 'SparkUI' on port 4040.
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@5ac7aa18{/jobs,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@13047d7d{/jobs/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@65bb9029{/jobs/job,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@49601f82{/jobs/job/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@2b8d084{/stages,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@24fabd0f{/stages/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@61f3fbb8{/stages/stage,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@60e5272{/stages/stage/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@69c93ca4{/stages/pool,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@173373b4{/stages/pool/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@60dd3c23{/storage,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@5e9456ae{/storage/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@1f1cae23{/storage/rdd,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@985696{/storage/rdd/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@215a34b4{/environment,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@35d3ab60{/environment/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@71870da7{/executors,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@45792847{/executors/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@4e25147a{/executors/threadDump,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@675ffd1d{/executors/threadDump/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@30506c0d{/static,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@771db12c{/,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@26ae880a{/api,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@5c645b43{/jobs/job/kill,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@298d9a05{/stages/stage/kill,null,AVAILABLE,@Spark}
   22/10/10 05:39:46 INFO SparkUI: Bound SparkUI to 0.0.0.0, and started at http://ip-10-224-51-45.ap-south-1.compute.internal:4040
   22/10/10 05:39:46 INFO SparkContext: Added JAR file:/usr/lib/hudi/cli/hudi-cli-0.10.1-amzn-0.jar at spark://ip-10-224-51-45.ap-south-1.compute.internal:37043/jars/hudi-cli-0.10.1-amzn-0.jar with timestamp 1665380385102
   22/10/10 05:39:46 INFO Executor: Starting executor ID driver on host ip-10-224-51-45.ap-south-1.compute.internal
   22/10/10 05:39:46 INFO Executor: Fetching spark://ip-10-224-51-45.ap-south-1.compute.internal:37043/jars/hudi-cli-0.10.1-amzn-0.jar with timestamp 1665380385102
   22/10/10 05:39:46 INFO TransportClientFactory: Successfully created connection to ip-10-224-51-45.ap-south-1.compute.internal/10.224.51.45:37043 after 29 ms (0 ms spent in bootstraps)
   22/10/10 05:39:46 INFO Utils: Fetching spark://ip-10-224-51-45.ap-south-1.compute.internal:37043/jars/hudi-cli-0.10.1-amzn-0.jar to /mnt/tmp/spark-52a5a695-a32e-4d74-bf11-83563425004c/userFiles-7dc5a66f-473c-4f03-bf5a-906c46e504d5/fetchFileTemp7148092357809489685.tmp
   22/10/10 05:39:46 INFO Executor: Adding file:/mnt/tmp/spark-52a5a695-a32e-4d74-bf11-83563425004c/userFiles-7dc5a66f-473c-4f03-bf5a-906c46e504d5/hudi-cli-0.10.1-amzn-0.jar to class loader
   22/10/10 05:39:46 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 43033.
   22/10/10 05:39:46 INFO NettyBlockTransferService: Server created on ip-10-224-51-45.ap-south-1.compute.internal:43033
   22/10/10 05:39:46 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy
   22/10/10 05:39:46 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 43033, None)
   22/10/10 05:39:46 INFO BlockManagerMasterEndpoint: Registering block manager ip-10-224-51-45.ap-south-1.compute.internal:43033 with 912.3 MiB RAM, BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 43033, None)
   22/10/10 05:39:46 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 43033, None)
   22/10/10 05:39:46 INFO BlockManager: external shuffle service port = 7337
   22/10/10 05:39:46 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 43033, None)
   22/10/10 05:39:46 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@75a118e6{/metrics/json,null,AVAILABLE,@Spark}
   22/10/10 05:39:47 INFO ClientConfigurationFactory: Set initial getObject socket timeout to 2000 ms.
   22/10/10 05:39:47 INFO log: Logging initialized @4588ms to org.eclipse.jetty.util.log.Slf4jLog
   22/10/10 05:39:48 INFO Javalin: 
              __                      __ _
             / /____ _ _   __ ____ _ / /(_)____
        __  / // __ `/| | / // __ `// // // __ \
       / /_/ // /_/ / | |/ // /_/ // // // / / /
       \____/ \__,_/  |___/ \__,_//_//_//_/ /_/
   hudi:customer_event_offline_v1->
           https://javalin.io/documentation
   hudi:customer_event_offline_v1->
   22/10/10 05:39:48 INFO Javalin: Starting Javalin ...
   22/10/10 05:39:48 INFO Server: jetty-9.4.43.v20210629; built: 2021-06-30T11:07:22.254Z; git: 526006ecfa3af7f1a27ef3a288e2bef7ea9dd7e8; jvm 1.8.0_342-b07
   22/10/10 05:39:48 INFO Server: Started @5001ms
   22/10/10 05:39:48 INFO Javalin: Listening on http://localhost:36465/
   22/10/10 05:39:48 INFO Javalin: Javalin started in 180ms \o/
   22/10/10 05:39:49 INFO S3NativeFileSystem: Opening 's3://test-spark-hudi/test_campaign_event_offline_compact_v1/.hoodie/hoodie.properties' for reading
   22/10/10 05:39:49 INFO AbstractConnector: Stopped Spark@ecfbe91{HTTP/1.1, (http/1.1)}{0.0.0.0:4040}
   22/10/10 05:39:49 INFO SparkUI: Stopped Spark web UI at http://ip-10-224-51-45.ap-south-1.compute.internal:4040
   22/10/10 05:39:49 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped!
   22/10/10 05:39:49 INFO MemoryStore: MemoryStore cleared
   22/10/10 05:39:49 INFO BlockManager: BlockManager stopped
   22/10/10 05:39:49 INFO BlockManagerMaster: BlockManagerMaster stopped
   22/10/10 05:39:49 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped!
   22/10/10 05:39:50 INFO SparkContext: Successfully stopped SparkContext
   22/10/10 05:39:50 INFO ShutdownHookManager: Shutdown hook called
   22/10/10 05:39:50 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-fe542e3b-8ab1-468a-b8af-cfa58eef245c
   22/10/10 05:39:50 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-52a5a695-a32e-4d74-bf11-83563425004c
   **Attempted to schedule compaction for 20221010053942582**
   
   
   hudi:customer_event_offline_v1->compaction run --parallelism 2 --schemaFilePath "s3://test-spark-hudi/schema/offline_compact.avsc" --compactionInstant 20221010053942582
   22/10/10 05:40:14 INFO SparkContext: Running Spark version 3.2.0-amzn-0
   22/10/10 05:40:14 INFO ResourceUtils: ==============================================================
   22/10/10 05:40:14 INFO ResourceUtils: No custom resources configured for spark.driver.
   22/10/10 05:40:14 INFO ResourceUtils: ==============================================================
   22/10/10 05:40:14 INFO SparkContext: Submitted application: hoodie-cli-COMPACT_RUN
   22/10/10 05:40:14 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 4, script: , vendor: , memory -> name: memory, amount: 4096, script: , vendor: , offHeap -> name: offHeap, amount: 0, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0)
   22/10/10 05:40:14 INFO ResourceProfile: Limiting resource is cpus at 4 tasks per executor
   22/10/10 05:40:14 INFO ResourceProfileManager: Added ResourceProfile id: 0
   22/10/10 05:40:14 INFO SecurityManager: Changing view acls to: hadoop
   22/10/10 05:40:14 INFO SecurityManager: Changing modify acls to: hadoop
   22/10/10 05:40:14 INFO SecurityManager: Changing view acls groups to: 
   22/10/10 05:40:14 INFO SecurityManager: Changing modify acls groups to: 
   22/10/10 05:40:14 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(hadoop); groups with view permissions: Set(); users  with modify permissions: Set(hadoop); groups with modify permissions: Set()
   22/10/10 05:40:14 INFO deprecation: mapred.output.compression.codec is deprecated. Instead, use mapreduce.output.fileoutputformat.compress.codec
   22/10/10 05:40:14 INFO deprecation: mapred.output.compression.type is deprecated. Instead, use mapreduce.output.fileoutputformat.compress.type
   22/10/10 05:40:14 INFO deprecation: mapred.output.compress is deprecated. Instead, use mapreduce.output.fileoutputformat.compress
   22/10/10 05:40:14 INFO Utils: Successfully started service 'sparkDriver' on port 34943.
   22/10/10 05:40:14 INFO SparkEnv: Registering MapOutputTracker
   22/10/10 05:40:14 INFO SparkEnv: Registering BlockManagerMaster
   22/10/10 05:40:14 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information
   22/10/10 05:40:14 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up
   22/10/10 05:40:14 INFO SparkEnv: Registering BlockManagerMasterHeartbeat
   22/10/10 05:40:14 INFO DiskBlockManager: Created local directory at /mnt/tmp/blockmgr-a0fd4166-022f-44a5-9709-33e6e390c281
   22/10/10 05:40:14 INFO MemoryStore: MemoryStore started with capacity 912.3 MiB
   22/10/10 05:40:14 INFO SparkEnv: Registering OutputCommitCoordinator
   22/10/10 05:40:14 INFO SubResultCacheManager: Sub-result caches are disabled.
   22/10/10 05:40:14 INFO log: Logging initialized @2733ms to org.sparkproject.jetty.util.log.Slf4jLog
   22/10/10 05:40:14 INFO Server: jetty-9.4.43.v20210629; built: 2021-06-30T11:07:22.254Z; git: 526006ecfa3af7f1a27ef3a288e2bef7ea9dd7e8; jvm 1.8.0_342-b07
   22/10/10 05:40:14 INFO Server: Started @2841ms
   22/10/10 05:40:14 INFO AbstractConnector: Started ServerConnector@150466c4{HTTP/1.1, (http/1.1)}{0.0.0.0:4040}
   22/10/10 05:40:14 INFO Utils: Successfully started service 'SparkUI' on port 4040.
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@566d0c69{/jobs,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@bdc8014{/jobs/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@73ba6fe6{/jobs/job,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@87abc48{/jobs/job/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@782168b7{/stages,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@7435a578{/stages/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@13047d7d{/stages/stage,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@2b214b94{/stages/stage/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@49601f82{/stages/pool,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@2b8d084{/stages/pool/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@24fabd0f{/storage,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@61f3fbb8{/storage/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@432034a{/storage/rdd,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@60e5272{/storage/rdd/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@69c93ca4{/environment,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@173373b4{/environment/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@60dd3c23{/executors,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@5e9456ae{/executors/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@1f1cae23{/executors/threadDump,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@985696{/executors/threadDump/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@215a34b4{/static,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@14fc5d40{/,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@47d7bfb3{/api,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@5f13be1{/jobs/job/kill,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@50d3bf39{/stages/stage/kill,null,AVAILABLE,@Spark}
   22/10/10 05:40:15 INFO SparkUI: Bound SparkUI to 0.0.0.0, and started at http://ip-10-224-51-45.ap-south-1.compute.internal:4040
   22/10/10 05:40:15 INFO SparkContext: Added JAR file:/usr/lib/hudi/cli/hudi-cli-0.10.1-amzn-0.jar at spark://ip-10-224-51-45.ap-south-1.compute.internal:34943/jars/hudi-cli-0.10.1-amzn-0.jar with timestamp 1665380413987
   22/10/10 05:40:15 INFO Executor: Starting executor ID driver on host ip-10-224-51-45.ap-south-1.compute.internal
   22/10/10 05:40:15 INFO Executor: Fetching spark://ip-10-224-51-45.ap-south-1.compute.internal:34943/jars/hudi-cli-0.10.1-amzn-0.jar with timestamp 1665380413987
   22/10/10 05:40:15 INFO TransportClientFactory: Successfully created connection to ip-10-224-51-45.ap-south-1.compute.internal/10.224.51.45:34943 after 29 ms (0 ms spent in bootstraps)
   22/10/10 05:40:15 INFO Utils: Fetching spark://ip-10-224-51-45.ap-south-1.compute.internal:34943/jars/hudi-cli-0.10.1-amzn-0.jar to /mnt/tmp/spark-1a761ac7-6903-41e9-8c8b-8d591f36d810/userFiles-1cc44228-430f-4503-a49c-77d959ead06b/fetchFileTemp3032838916835851411.tmp
   22/10/10 05:40:15 INFO Executor: Adding file:/mnt/tmp/spark-1a761ac7-6903-41e9-8c8b-8d591f36d810/userFiles-1cc44228-430f-4503-a49c-77d959ead06b/hudi-cli-0.10.1-amzn-0.jar to class loader
   22/10/10 05:40:15 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 46483.
   22/10/10 05:40:15 INFO NettyBlockTransferService: Server created on ip-10-224-51-45.ap-south-1.compute.internal:46483
   22/10/10 05:40:15 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy
   22/10/10 05:40:15 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 46483, None)
   22/10/10 05:40:15 INFO BlockManagerMasterEndpoint: Registering block manager ip-10-224-51-45.ap-south-1.compute.internal:46483 with 912.3 MiB RAM, BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 46483, None)
   22/10/10 05:40:15 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 46483, None)
   22/10/10 05:40:15 INFO BlockManager: external shuffle service port = 7337
   22/10/10 05:40:15 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, ip-10-224-51-45.ap-south-1.compute.internal, 46483, None)
   22/10/10 05:40:15 INFO ContextHandler: Started o.s.j.s.ServletContextHandler@61d84e08{/metrics/json,null,AVAILABLE,@Spark}
   22/10/10 05:40:16 INFO ClientConfigurationFactory: Set initial getObject socket timeout to 2000 ms.
   22/10/10 05:40:17 INFO S3NativeFileSystem: Opening 's3://test-spark-hudi/schema/offline_compact.avsc' for reading
   22/10/10 05:40:17 INFO log: Logging initialized @5802ms to org.eclipse.jetty.util.log.Slf4jLog
   22/10/10 05:40:17 INFO Javalin: 
              __                      __ _
             / /____ _ _   __ ____ _ / /(_)____
        __  / // __ `/| | / // __ `// // // __ \
       / /_/ // /_/ / | |/ // /_/ // // // / / /
       \____/ \__,_/  |___/ \__,_//_//_//_/ /_/
   hudi:customer_event_offline_v1->
           https://javalin.io/documentation
   hudi:customer_event_offline_v1->
   22/10/10 05:40:17 INFO Javalin: Starting Javalin ...
   22/10/10 05:40:18 INFO Server: jetty-9.4.43.v20210629; built: 2021-06-30T11:07:22.254Z; git: 526006ecfa3af7f1a27ef3a288e2bef7ea9dd7e8; jvm 1.8.0_342-b07
   22/10/10 05:40:18 INFO Server: Started @6068ms
   22/10/10 05:40:18 INFO Javalin: Listening on http://localhost:42971/
   22/10/10 05:40:18 INFO Javalin: Javalin started in 162ms \o/
   22/10/10 05:40:18 INFO S3NativeFileSystem: Opening 's3://test-spark-hudi/test_campaign_event_offline_compact_v1/.hoodie/hoodie.properties' for reading
   22/10/10 05:40:18 INFO S3NativeFileSystem: Opening 's3://test-spark-hudi/test_campaign_event_offline_compact_v1/.hoodie/20221007132341606.deltacommit' for reading
   22/10/10 05:40:19 ERROR UtilHelpers: Compact failed
   **java.lang.IllegalStateException: No Compaction request available at 20221010053942582 to run compaction
   hudi:cusat** org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.preCompact(HoodieSparkMergeOnReadTableCompactor.java:49)
   hudi:cusat org.apache.hudi.table.action.compact.RunCompactionActionExecutor.execute(RunCompactionActionExecutor.java:64)
   hudi:cusat org.apache.hudi.table.HoodieSparkMergeOnReadTable.compact(HoodieSparkMergeOnReadTable.java:143)
   hudi:cusat org.apache.hudi.client.SparkRDDWriteClient.compact(SparkRDDWriteClient.java:341)
   hudi:cusat org.apache.hudi.client.SparkRDDWriteClient.compact(SparkRDDWriteClient.java:75)
   hudi:cusat org.apache.hudi.client.AbstractHoodieWriteClient.compact(AbstractHoodieWriteClient.java:860)
   hudi:cusat org.apache.hudi.utilities.HoodieCompactor.doCompact(HoodieCompactor.java:156)
   hudi:cusat org.apache.hudi.utilities.HoodieCompactor.lambda$compact$0(HoodieCompactor.java:130)
   hudi:cusat org.apache.hudi.utilities.UtilHelpers.retry(UtilHelpers.java:488)
   hudi:cusat org.apache.hudi.utilities.HoodieCompactor.compact(HoodieCompactor.java:123)
   hudi:cusat org.apache.hudi.cli.commands.SparkMain.compact(SparkMain.java:336)
   hudi:cusat org.apache.hudi.cli.commands.SparkMain.main(SparkMain.java:130)
   hudi:cusat sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   hudi:cusat sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   hudi:cusat sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   hudi:cusat java.lang.reflect.Method.invoke(Method.java:498)
   hudi:cusat org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
   hudi:cusat org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1000)
   hudi:cusat org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
   hudi:cusat org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
   hudi:cusat org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
   hudi:cusat org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1089)
   hudi:cusat org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1098)
   hudi:cusat org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   22/10/10 05:40:19 INFO AbstractConnector: Stopped Spark@150466c4{HTTP/1.1, (http/1.1)}{0.0.0.0:4040}
   22/10/10 05:40:19 INFO SparkUI: Stopped Spark web UI at http://ip-10-224-51-45.ap-south-1.compute.internal:4040
   22/10/10 05:40:19 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped!
   22/10/10 05:40:19 INFO MemoryStore: MemoryStore cleared
   22/10/10 05:40:19 INFO BlockManager: BlockManager stopped
   22/10/10 05:40:19 INFO BlockManagerMaster: BlockManagerMaster stopped
   22/10/10 05:40:19 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped!
   22/10/10 05:40:19 INFO SparkContext: Successfully stopped SparkContext
   22/10/10 05:40:19 INFO ShutdownHookManager: Shutdown hook called
   22/10/10 05:40:19 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-1a761ac7-6903-41e9-8c8b-8d591f36d810
   22/10/10 05:40:19 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-8c125f58-c509-466f-9c47-e2d2a4718540
   Failed to run compaction for 20221010053942582
   ```
   
   


-- 
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: commits-unsubscribe@hudi.apache.org.apache.org

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


[GitHub] [hudi] maheshguptags commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
maheshguptags commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1278645561

   @yihua I tried without `--mode ` parameter and it is not scheduling the compaction.  


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] maheshguptags commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
maheshguptags commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273322669

   @KnightChess same error
   ```
   spark-submit  --class org.apache.hudi.utilities.HoodieCompactor --jars /usr/lib/hudi/hudi-spark3-bundle_2.12-0.10.1-amzn-0.jar /usr/lib/hudi/hudi-utilities-bundle_2.12-0.10.1-amzn-0.jar --base-path "s3://test-spark-hudi/test_campaign_event_offline_compact_v1_sch/" --table-name "customer_event_offline_v1_sch" --schema-file "s3://test-spark-hudi/schema/offline_compact.avsc" --mode schedule --strategy "org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy" --instant-time "20221007120816651"  --spark-memory 1g --parallelism 2 
   Exception in thread "main" org.apache.hudi.com.beust.jcommander.ParameterException: Was passed main parameter '--mode' but no main parameter was defined in your arg class
   	at org.apache.hudi.com.beust.jcommander.JCommander.initMainParameterValue(JCommander.java:936)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parseValues(JCommander.java:752)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parse(JCommander.java:340)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parse(JCommander.java:319)
   	at org.apache.hudi.com.beust.jcommander.JCommander.<init>(JCommander.java:240)
   	at org.apache.hudi.utilities.HoodieCompactor.main(HoodieCompactor.java:105)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
   	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1000)
   	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
   	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
   	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
   	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1089)
   	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1098)
   	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   22/10/10 13:34:37 INFO ShutdownHookManager: Shutdown hook called
   22/10/10 13:34:37 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-79edaac1-38b8-436d-a2f3-d82fcbf6ff20
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] maheshguptags commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
maheshguptags commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1272818509

   CC : 
   @bhasudha 
   @codope 
   @nsivabalan 


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
KnightChess commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273267737

   Can you have a try use `--mode schedule`


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] yihua commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1278615104

   @maheshguptags could you remove the `--mode schedule` parameter?  `--mode` is introduced in 0.11.0.  Given that you're using 0.10.1, this parameter is not available.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] maheshguptags commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
maheshguptags commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273309934

   Same Error @KnightChess 
   spark-submit  --class org.apache.hudi.utilities.HoodieCompactor --jars /usr/lib/hudi/hudi-spark3-bundle_2.12-0.10.1-amzn-0.jar /usr/lib/hudi/hudi-utilities-bundle_2.12-0.10.1-amzn-0.jar --base-path "s3://test-spark-hudi/test_campaign_event_offline_compact_v1_sch/" --table-name "customer_event_offline_v1_sch" --schema-file "s3://test-spark-hudi/schema/offline_compact.avsc" --mode "schedule" --strategy "org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy" --instant-time "20221007120816651"  --spark-memory 1g --parallelism 2 
   
   Exception in thread "main" org.apache.hudi.com.beust.jcommander.ParameterException: Was passed main parameter '--mode' but no main parameter was defined in your arg class
   	at org.apache.hudi.com.beust.jcommander.JCommander.initMainParameterValue(JCommander.java:936)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parseValues(JCommander.java:752)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parse(JCommander.java:340)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parse(JCommander.java:319)
   	at org.apache.hudi.com.beust.jcommander.JCommander.<init>(JCommander.java:240)
   	at org.apache.hudi.utilities.HoodieCompactor.main(HoodieCompactor.java:105)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
   	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1000)
   	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
   	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
   	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
   	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1089)
   	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1098)
   	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   22/10/10 13:24:31 INFO ShutdownHookManager: Shutdown hook called
   22/10/10 13:24:31 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-f12c1514-49c5-4d64-a9d3-379ba4bbd609
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
KnightChess commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273320443

   can you remove double quotes in mode param, in my case it can work
   <img width="776" alt="image" src="https://user-images.githubusercontent.com/20125927/194878069-4155f864-ccc9-41b7-befe-052384cf13d3.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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
KnightChess commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273076513

   look like running mode is `execute`, maybe param `--schedule` need a value `true` like `--schedule true`


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan closed issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #6903: Offline compaction scheduling not working 
URL: https://github.com/apache/hudi/issues/6903


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] maheshguptags commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
maheshguptags commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1274426723

   @nsivabalan still getting same error 
   ```$ spark-submit --jars /usr/lib/hudi/hudi-spark3-bundle_2.12-0.10.1-amzn-0.jar --class org.apache.hudi.utilities.HoodieCompactor /usr/lib/hudi/hudi-utilities-bundle_2.12-0.10.1-amzn-0.jar  --base-path "s3://test-spark-hudi/test_campaign_event_offline_compact_v1_sch/" --table-name "customer_event_offline_v1_sch" --schema-file "s3://test-spark-hudi/schema/offline_compact.avsc" --mode schedule --strategy "org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy" --instant-time "20221007120816651"  --spark-memory 1g --parallelism 2 
   Exception in thread "main" org.apache.hudi.com.beust.jcommander.ParameterException: Was passed main parameter '--mode' but no main parameter was defined in your arg class
   	at org.apache.hudi.com.beust.jcommander.JCommander.initMainParameterValue(JCommander.java:936)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parseValues(JCommander.java:752)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parse(JCommander.java:340)
   	at org.apache.hudi.com.beust.jcommander.JCommander.parse(JCommander.java:319)
   	at org.apache.hudi.com.beust.jcommander.JCommander.<init>(JCommander.java:240)
   	at org.apache.hudi.utilities.HoodieCompactor.main(HoodieCompactor.java:105)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
   	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1000)
   	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
   	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
   	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
   	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1089)
   	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1098)
   	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   22/10/11 09:48:17 INFO ShutdownHookManager: Shutdown hook called
   22/10/11 09:48:17 INFO ShutdownHookManager: Deleting directory /mnt/tmp/spark-669bb840-3e41-4957-a6ce-46f265bb9f35
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1287875616

   closing the issue. feel free to re-open if you need any more assistance. 


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1283320460

   I used 0.10.1 for above. 


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1283269668

   @maheshguptags :  whats the stacktrace you see w/o --mode? 


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1283320289

   found time to reproduce. 
   we need to do it in 2 commands. 
   1: schedule
   2: execute
   
   
   sample commands I used
   1. to schedule
   ```
   ./bin/spark-submit  --class org.apache.hudi.utilities.HoodieCompactor ~/Documents/personal/projects/apache_hudi_dec/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.10.1.jar  --base-path "/tmp/hudi_trips_cow" --table-name "hudi_trips_cow" --schema-file "/tmp/compact.avsc"  --strategy "org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy" --instant-time "20221018192633991"  --spark-memory 1g --parallelism 2 --schedule --hoodie-conf "hoodie.compact.inline.max.delta.commits=2"
   ```
   
   configs of interest
   ```
   --schedule --hoodie-conf "hoodie.compact.inline.max.delta.commits=2"
   ```
   
   2. to execute 
   ```
   ./bin/spark-submit  --class org.apache.hudi.utilities.HoodieCompactor ~/Documents/personal/projects/apache_hudi_dec/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.10.1.jar  --base-path "/tmp/hudi_trips_cow" --table-name "hudi_trips_cow" --schema-file "/tmp/compact.avsc"  --strategy "org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy" --instant-time "20221018192633991"  --spark-memory 1g --parallelism 2
   ```


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] maheshguptags commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
maheshguptags commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273081129

   @KnightChess thanks for looking into the issue and I tried with the parameter that you mentioned above but same error persist. 


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on issue #6903: Offline compaction scheduling not working

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #6903:
URL: https://github.com/apache/hudi/issues/6903#issuecomment-1273914016

   likely the issue is, placement of `--jars`
   
   ```
   spark-submit --class org.apache.hudi.utilities.HoodieCompactor --jars /usr/lib/hudi/hudi-spark3-bundle_2.12-0.10.1-amzn-0.jar /usr/lib/hudi/hudi-utilities-bundle_2.12-0.10.1-amzn-0.jar
   ```
   
   any spark options should preced the class name and app jar 
   
   ```
   spark-submit --jars /usr/lib/hudi/hudi-spark3-bundle_2.12-0.10.1-amzn-0.jar --class org.apache.hudi.utilities.HoodieCompactor /usr/lib/hudi/hudi-utilities-bundle_2.12-0.10.1-amzn-0.jar
   ```
   
   also, utilties bundle includes spark. so you don't even need spark bundle to be passed in. 
   


-- 
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: commits-unsubscribe@hudi.apache.org

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