You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "Jonathan Vexler (Jira)" <ji...@apache.org> on 2023/03/15 15:01:00 UTC

[jira] [Commented] (HUDI-5891) Clustering fails on uncompacted bootstrapped mor table

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

Jonathan Vexler commented on HUDI-5891:
---------------------------------------

Even with compaction enabled, I was still able to trigger this exception:
{code:java}
drwxr-xr-x  4 jon  staff   128B Mar  7 14:00 metadata
drwxr-xr-x  2 jon  staff    64B Mar  7 14:00 archived
-rw-r--r--  1 jon  staff   1.2K Mar  7 14:00 hoodie.properties
-rw-r--r--  1 jon  staff     0B Mar  7 14:00 00000000000001.deltacommit.requested
-rw-r--r--  1 jon  staff     0B Mar  7 14:00 00000000000001.deltacommit.inflight
-rw-r--r--  1 jon  staff     0B Mar  7 14:01 00000000000002.deltacommit.requested
-rw-r--r--  1 jon  staff     0B Mar  7 14:01 00000000000002.deltacommit.inflight
-rw-r--r--  1 jon  staff    24K Mar  7 14:01 00000000000001.deltacommit
-rw-r--r--  1 jon  staff    24K Mar  7 14:01 00000000000002.deltacommit
-rw-r--r--  1 jon  staff     0B Mar  7 18:00 20230307180048112.deltacommit.requested
-rw-r--r--  1 jon  staff    42K Mar  7 18:01 20230307180048112.deltacommit.inflight
-rw-r--r--  1 jon  staff    59K Mar  7 18:01 20230307180048112.deltacommit
-rw-r--r--  1 jon  staff    20K Mar  7 18:01 20230307180131715.compaction.requested
-rw-r--r--  1 jon  staff     0B Mar  7 18:01 20230307180131715.compaction.inflight
-rw-r--r--  1 jon  staff    39K Mar  7 18:01 20230307180131715.commit
-rw-r--r--  1 jon  staff    25K Mar  7 18:01 20230307180137876.replacecommit.requested
-rw-r--r--  1 jon  staff     0B Mar  7 18:01 20230307180137876.replacecommit.inflight {code}
Exception:
{code:java}
java.util.concurrent.CompletionException: java.lang.IllegalArgumentException: Partition path does not belong to base-path
  at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
  at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
  at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1606)
  at java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1596)
  at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
  at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
  at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
  at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
Caused by: java.lang.IllegalArgumentException: Partition path does not belong to base-path
  at org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath(FSUtils.java:228)
  at org.apache.hudi.BaseHoodieTableFileIndex.lambda$getAllQueryPartitionPaths$0(BaseHoodieTableFileIndex.java:203)
  at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
  at java.util.Iterator.forEachRemaining(Iterator.java:116)
  at scala.collection.convert.Wrappers$IteratorWrapper.forEachRemaining(Wrappers.scala:31)
  at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
  at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
  at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
  at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
  at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
  at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
  at org.apache.hudi.BaseHoodieTableFileIndex.getAllQueryPartitionPaths(BaseHoodieTableFileIndex.java:204)
  at org.apache.hudi.SparkHoodieTableFileIndex.listMatchingPartitionPaths(SparkHoodieTableFileIndex.scala:205)
  at org.apache.hudi.HoodieFileIndex.listFiles(HoodieFileIndex.scala:146)
  at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions$$anonfun$apply$1.applyOrElse(HoodiePruneFileSourcePartitions.scala:54)
  at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions$$anonfun$apply$1.applyOrElse(HoodiePruneFileSourcePartitions.scala:42)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:584)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:584)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:589)
  at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1228)
  at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1227)
  at org.apache.spark.sql.catalyst.plans.logical.Sort.mapChildren(basicLogicalOperators.scala:755)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:589)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:589)
  at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1228)
  at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1227)
  at org.apache.spark.sql.catalyst.plans.logical.RepartitionOperation.mapChildren(basicLogicalOperators.scala:1429)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:589)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:560)
  at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions.apply(HoodiePruneFileSourcePartitions.scala:42)
  at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions.apply(HoodiePruneFileSourcePartitions.scala:40)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:211)
  at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
  at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
  at scala.collection.immutable.List.foldLeft(List.scala:91)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:208)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200)
  at scala.collection.immutable.List.foreach(List.scala:431)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:126)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:185)
  at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:510)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:185)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
  at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:184)
  at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:122)
  at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:118)
  at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:136)
  at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:154)
  at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:151)
  at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:173)
  at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:172)
  at org.apache.hudi.HoodieDatasetBulkInsertHelper$.bulkInsert(HoodieDatasetBulkInsertHelper.scala:142)
  at org.apache.hudi.HoodieDatasetBulkInsertHelper.bulkInsert(HoodieDatasetBulkInsertHelper.scala)
  at org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy.performClusteringWithRecordsAsRow(SparkSortAndSizeExecutionStrategy.java:72)
  at org.apache.hudi.client.clustering.run.strategy.MultipleSparkJobExecutionStrategy.lambda$runClusteringForGroupAsyncAsRow$6(MultipleSparkJobExecutionStrategy.java:249)
  at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
  ... 5 more {code}

> Clustering fails on uncompacted bootstrapped mor table
> ------------------------------------------------------
>
>                 Key: HUDI-5891
>                 URL: https://issues.apache.org/jira/browse/HUDI-5891
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: bootstrap, clustering, compaction
>            Reporter: Jonathan Vexler
>            Priority: Major
>
> If you create a bootstrapped table and then write upserts with the configs below, it will fail when clustering is attempted. On a non-bootstrap table, clustering will succeed.
> {code:java}
> betterdf.write.format("hudi")
>             .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
>             .option(DataSourceWriteOptions.OPERATION_OPT_KEY, "upsert")
>             .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "key")
>             .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partition")
>             .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts")
>             .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key, "true")
>             .option(DataSourceWriteOptions.TABLE_TYPE.key, tt)
>             .option("hoodie.compact.inline", "false")
> .option("hoodie.clustering.inline", "true")             .option("hoodie.clustering.plan.strategy.sort.columns", "partition,key")
>             .option("hoodie.index.type", "MERGE_ON_READ")
>             .mode(SaveMode.Append)
>             .save(basePath) {code}
> When clustering is attempted I get the exception:
> {code:java}
> java.util.concurrent.CompletionException: java.lang.IllegalArgumentException: Partition path does not belong to base-path
>   at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
>   at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
>   at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1606)
>   at java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1596)
>   at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
>   at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
>   at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
>   at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
> Caused by: java.lang.IllegalArgumentException: Partition path does not belong to base-path
>   at org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath(FSUtils.java:228)
>   at org.apache.hudi.BaseHoodieTableFileIndex.lambda$getAllQueryPartitionPaths$0(BaseHoodieTableFileIndex.java:203)
>   at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>   at java.util.Iterator.forEachRemaining(Iterator.java:116)
>   at scala.collection.convert.Wrappers$IteratorWrapper.forEachRemaining(Wrappers.scala:31)
>   at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
>   at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>   at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>   at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
>   at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>   at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
>   at org.apache.hudi.BaseHoodieTableFileIndex.getAllQueryPartitionPaths(BaseHoodieTableFileIndex.java:204)
>   at org.apache.hudi.SparkHoodieTableFileIndex.listMatchingPartitionPaths(SparkHoodieTableFileIndex.scala:205)
>   at org.apache.hudi.HoodieFileIndex.listFiles(HoodieFileIndex.scala:146)
>   at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions$$anonfun$apply$1.applyOrElse(HoodiePruneFileSourcePartitions.scala:54)
>   at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions$$anonfun$apply$1.applyOrElse(HoodiePruneFileSourcePartitions.scala:42)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:584)
>   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:584)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:589)
>   at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1228)
>   at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1227)
>   at org.apache.spark.sql.catalyst.plans.logical.Sort.mapChildren(basicLogicalOperators.scala:755)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:589)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:589)
>   at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1228)
>   at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1227)
>   at org.apache.spark.sql.catalyst.plans.logical.RepartitionOperation.mapChildren(basicLogicalOperators.scala:1429)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:589)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:560)
>   at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions.apply(HoodiePruneFileSourcePartitions.scala:42)
>   at org.apache.spark.sql.hudi.analysis.HoodiePruneFileSourcePartitions.apply(HoodiePruneFileSourcePartitions.scala:40)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:211)
>   at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
>   at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
>   at scala.collection.immutable.List.foldLeft(List.scala:91)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:208)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200)
>   at scala.collection.immutable.List.foreach(List.scala:431)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179)
>   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179)
>   at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:126)
>   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
>   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:185)
>   at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:510)
>   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:185)
>   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
>   at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:184)
>   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:122)
>   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:118)
>   at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:136)
>   at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:154)
>   at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:151)
>   at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:173)
>   at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:172)
>   at org.apache.hudi.HoodieDatasetBulkInsertHelper$.bulkInsert(HoodieDatasetBulkInsertHelper.scala:142)
>   at org.apache.hudi.HoodieDatasetBulkInsertHelper.bulkInsert(HoodieDatasetBulkInsertHelper.scala)
>   at org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy.performClusteringWithRecordsAsRow(SparkSortAndSizeExecutionStrategy.java:72)
>   at org.apache.hudi.client.clustering.run.strategy.MultipleSparkJobExecutionStrategy.lambda$runClusteringForGroupAsyncAsRow$6(MultipleSparkJobExecutionStrategy.java:249)
>   at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
>   ... 5 more {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)