voonhous opened a new issue, #7663: URL: https://github.com/apache/hudi/issues/7663
**_Tips before filing an issue_** - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)? - Join the mailing list to engage in conversations and get faster support at [email protected]. - If you have triaged this as a bug, then file an [issue](https://issues.apache.org/jira/projects/HUDI/issues) directly. **Describe the problem you faced** A clear and concise description of the problem. # TLDR Dropping a partition when there's a pending table service action may cause downstream write errors + data inconsistencies # Detailed explanation 1. There's an append-only job with online clustering enabled that is executing. 2. User executed a ALTER-TABLE DROP partition DDL to drop a partition 3. User is still able to query out the partition's data 4. Operations on a table may fail if `AbstractTableFileSystemView#resetFileGroupsReplaced` is invoked # Root cause 1. clustering plan created to cluster `partition=a/filegroup_1` (assume that `partition=a` only has `filegroup_1`) 2. User executed a ALTER-TABLE DROP partition DDL to drop `partition=a` 3. replacecommit file created for ALTER-TABLE DROP PARTITION DDL to replace/ignore `filegroup_1` 5. clustering plan is executed, at this point, it will cluster `partition=a/filegroup_1` to produce `partition=a/filegroup_2` 6. when reading a partition, it will read out `partition=a/filegroup_2`. # Errors 1. Spark-SQL will still show data in the partition 2. When FileSystemView is invoked, an error like this will be thrown: **To Reproduce** Steps to reproduce the behavior: ```scala test("ALTER TABLE DROP PARTITION + CLUSTER (Unable to drop partition issue)") { withTempDir { tmp => Seq("cow").foreach { tableType => val tableName = generateTableName val basePath = s"${tmp.getCanonicalPath}t/$tableName" spark.sql( s""" |create table $tableName ( | id int, | name string, | price double, | ts long |) using hudi | options ( | primaryKey ='id', | type = '$tableType', | preCombineField = 'ts' | ) | partitioned by(ts) | location '$basePath' """.stripMargin) spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) // Generate the first clustering plan val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty()) checkAnswer(s"call show_clustering('$tableName')")( Seq(firstScheduleInstant, 3, HoodieInstant.State.REQUESTED.name(), "*") ) spark.sql(s"ALTER TABLE $tableName DROP PARTITION(ts=1002)") // Do clustering for all clustering plan generated above, and no new clustering // instant will be generated because of there is no commit after the second // clustering plan generated checkAnswer(s"call run_clustering(table => '$tableName', order => 'ts', show_involved_partition => true)")( Seq(firstScheduleInstant, 3, HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001,ts=1002") ) // Check the number of finished clustering instants val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) val finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) .getInstants .iterator().asScala .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) .toSeq // 1 drop partition + 1 clustering assertResult(2)(finishedClustering.size) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 10.0, 1000), Seq(2, "a2", 10.0, 1001) ) } } ``` **Expected behavior** A clear and concise description of what you expected to happen. **Environment Description** * Hudi version : 0.13.0 * Spark version : N.A * Hive version : N.A * Hadoop version : N.A * Storage (HDFS/S3/GCS..) : * Running on Docker? (yes/no) : **Additional context** Add any other context about the problem here. **Stacktrace** ```Add the stacktrace of the error.``` ```log 2023-01-11 10:19:47,281 ERROR org.apache.hudi.sink.StreamWriteOperatorCoordinator [] - Executor executes action [commits the instant 20230111101628266] error org.apache.hudi.exception.HoodieException: org.apache.hudi.exception.HoodieException: Error occurs when executing flatMap at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) ~[?:1.8.0_252] at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) ~[?:1.8.0_252] at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) ~[?:1.8.0_252] at java.lang.reflect.Constructor.newInstance(Constructor.java:423) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinTask.getThrowableException(ForkJoinTask.java:593) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinTask.reportException(ForkJoinTask.java:677) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinTask.invoke(ForkJoinTask.java:735) ~[?:1.8.0_252] at java.util.stream.ReduceOps$ReduceOp.evaluateParallel(ReduceOps.java:714) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:233) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566) ~[?:1.8.0_252] at org.apache.hudi.client.common.HoodieFlinkEngineContext.flatMap(HoodieFlinkEngineContext.java:135) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy.generateClusteringPlan(PartitionAwareClusteringPlanStrategy.java:87) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor.createClusteringPlan(ClusteringPlanActionExecutor.java:86) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor.execute(ClusteringPlanActionExecutor.java:91) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.scheduleClustering(HoodieFlinkCopyOnWriteTable.java:285) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.client.BaseHoodieWriteClient.scheduleTableServiceInternal(BaseHoodieWriteClient.java:1347) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.client.BaseHoodieWriteClient.scheduleTableService(BaseHoodieWriteClient.java:1329) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.client.BaseHoodieWriteClient.scheduleClusteringAtInstant(BaseHoodieWriteClient.java:1274) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.client.BaseHoodieWriteClient.scheduleClustering(BaseHoodieWriteClient.java:1265) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.util.ClusteringUtil.scheduleClustering(ClusteringUtil.java:58) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$notifyCheckpointComplete$2(StreamWriteOperatorCoordinator.java:253) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130) ~[hudi.jar:0.12.1-di01] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_252] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_252] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_252] Caused by: org.apache.hudi.exception.HoodieException: Error occurs when executing flatMap at org.apache.hudi.common.function.FunctionWrapper.lambda$throwingFlatMapWrapper$1(FunctionWrapper.java:50) ~[hudi.jar:0.12.1-di01] at java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:269) ~[?:1.8.0_252] at java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_252] at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:747) ~[?:1.8.0_252] at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:721) ~[?:1.8.0_252] at java.util.stream.AbstractTask.compute(AbstractTask.java:327) ~[?:1.8.0_252] at java.util.concurrent.CountedCompleter.exec(CountedCompleter.java:731) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157) ~[?:1.8.0_252] Caused by: java.lang.IllegalStateException: Duplicate key [20230111000227062__replacecommit__COMPLETED] at java.util.stream.Collectors.lambda$throwingMerger$0(Collectors.java:133) ~[?:1.8.0_252] at java.util.HashMap.merge(HashMap.java:1254) ~[?:1.8.0_252] at java.util.stream.Collectors.lambda$toMap$58(Collectors.java:1320) ~[?:1.8.0_252] at java.util.stream.ReduceOps$3ReducingSink.accept(ReduceOps.java:169) ~[?:1.8.0_252] at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) ~[?:1.8.0_252] at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) ~[?:1.8.0_252] at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_252] at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) ~[?:1.8.0_252] at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272) ~[?:1.8.0_252] at java.util.Iterator.forEachRemaining(Iterator.java:116) ~[?:1.8.0_252] at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_252] at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) ~[?:1.8.0_252] at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272) ~[?:1.8.0_252] at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_252] at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:1.8.0_252] at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566) ~[?:1.8.0_252] at org.apache.hudi.common.table.view.AbstractTableFileSystemView.resetFileGroupsReplaced(AbstractTableFileSystemView.java:242) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.table.view.AbstractTableFileSystemView.init(AbstractTableFileSystemView.java:108) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.table.view.HoodieTableFileSystemView.init(HoodieTableFileSystemView.java:108) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.table.view.HoodieTableFileSystemView.<init>(HoodieTableFileSystemView.java:102) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.table.view.FileSystemViewManager.createInMemoryFileSystemView(FileSystemViewManager.java:177) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.table.view.FileSystemViewManager.lambda$createViewManager$5fcdabfe$1(FileSystemViewManager.java:272) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.table.view.FileSystemViewManager.lambda$getFileSystemView$1(FileSystemViewManager.java:115) ~[hudi.jar:0.12.1-di01] at java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1660) ~[?:1.8.0_252] at org.apache.hudi.common.table.view.FileSystemViewManager.getFileSystemView(FileSystemViewManager.java:114) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.HoodieTable.getSliceView(HoodieTable.java:306) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy.getFileSlicesEligibleForClustering(ClusteringPlanStrategy.java:111) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.client.clustering.plan.strategy.FlinkSizeBasedClusteringPlanStrategy.getFileSlicesEligibleForClustering(FlinkSizeBasedClusteringPlanStrategy.java:116) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy.lambda$generateClusteringPlan$4e6aac78$1(PartitionAwareClusteringPlanStrategy.java:90) ~[hudi.jar:0.12.1-di01] at org.apache.hudi.common.function.FunctionWrapper.lambda$throwingFlatMapWrapper$1(FunctionWrapper.java:48) ~[hudi.jar:0.12.1-di01] at java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:269) ~[?:1.8.0_252] at java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_252] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_252] at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:747) ~[?:1.8.0_252] at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:721) ~[?:1.8.0_252] at java.util.stream.AbstractTask.compute(AbstractTask.java:327) ~[?:1.8.0_252] at java.util.concurrent.CountedCompleter.exec(CountedCompleter.java:731) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) ~[?:1.8.0_252] at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157) ~[?:1.8.0_252] ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
