lintingbin2009 opened a new issue, #8109:
URL: https://github.com/apache/iceberg/issues/8109

   ### Apache Iceberg version
   
   1.3.0 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   table schema:
   
   > CREATE TABLE spark_catalog.test.log(
     `log_type.string` STRING,
     `event_time.string` STRING,
     `version.bigint` BIGINT,
     `ip.string` STRING,
     `country.string` STRING)
   USING iceberg
   PARTITIONED BY (truncate(10, `event_time.string`), `log_type.string`)
   TBLPROPERTIES (
     'format-version' = '2',
     'read.parquet.vectorization.enabled' = 'true',
     'write.format.default' = 'parquet',
     'write.metadata.delete-after-commit.enabled' = 'true',
     'write.metadata.metrics.column.event_time.string' = 'truncate(16)',
     'write.metadata.metrics.default' = 'none',
     'write.metadata.previous-versions-max' = '10')
   
   When I execute call spark_catalog.system.rewrite_position_delete_files(table 
=> 'test.log'), the following error will be reported:
   
   > Caused by: org.apache.spark.sql.AnalysisException: No such struct field 
event_time in event_time.string_trunc, log_type.string
        at 
org.apache.spark.sql.errors.QueryCompilationErrors$.noSuchStructFieldInGivenFieldsError(QueryCompilationErrors.scala:1567)
        at 
org.apache.spark.sql.catalyst.expressions.ExtractValue$.findField(complexTypeExtractors.scala:82)
        at 
org.apache.spark.sql.catalyst.expressions.ExtractValue$.apply(complexTypeExtractors.scala:55)
        at 
org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.$anonfun$resolve$2(package.scala:357)
        at 
scala.collection.IndexedSeqOptimized.foldLeft(IndexedSeqOptimized.scala:60)
        at 
scala.collection.IndexedSeqOptimized.foldLeft$(IndexedSeqOptimized.scala:68)
        at scala.collection.mutable.ArrayBuffer.foldLeft(ArrayBuffer.scala:49)
        at 
org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:356)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:114)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$resolveExpressionByPlanChildren$1(Analyzer.scala:1802)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$resolveExpression$2(Analyzer.scala:1730)
        at 
org.apache.spark.sql.catalyst.analysis.package$.withPosition(package.scala:60)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.innerResolve$1(Analyzer.scala:1737)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$resolveExpression$8(Analyzer.scala:1752)
        at 
org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren(TreeNode.scala:1254)
        at 
org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren$(TreeNode.scala:1253)
        at 
org.apache.spark.sql.catalyst.expressions.BinaryExpression.mapChildren(Expression.scala:608)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.innerResolve$1(Analyzer.scala:1752)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$resolveExpression$8(Analyzer.scala:1752)
        at 
org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren(TreeNode.scala:1254)
        at 
org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren$(TreeNode.scala:1253)
        at 
org.apache.spark.sql.catalyst.expressions.BinaryExpression.mapChildren(Expression.scala:608)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.innerResolve$1(Analyzer.scala:1752)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.resolveExpression(Analyzer.scala:1757)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.resolveExpressionByPlanChildren(Analyzer.scala:1808)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$15.$anonfun$applyOrElse$78(Analyzer.scala:1516)
        at 
org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:200)
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176)
        at 
org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:200)
        at 
org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:211)
        at 
org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:221)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:427)
        at 
org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:221)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$15.applyOrElse(Analyzer.scala:1516)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$$anonfun$apply$15.applyOrElse(Analyzer.scala:1353)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138)
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:138)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:323)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:30)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.apply(Analyzer.scala:1353)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences$.apply(Analyzer.scala:1333)
        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.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:227)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:223)
        at 
org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:172)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:223)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:187)
        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.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:208)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:330)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:207)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:76)
        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.analyzed$lzycompute(QueryExecution.scala:76)
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:74)
        at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:66)
        at org.apache.spark.sql.Dataset.<init>(Dataset.scala:206)
        at org.apache.spark.sql.Dataset.<init>(Dataset.scala:212)
        at org.apache.spark.sql.Dataset$.apply(Dataset.scala:76)
        at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:3892)
        at org.apache.spark.sql.Dataset.filter(Dataset.scala:1653)
        at 
org.apache.iceberg.spark.actions.SparkBinPackPositionDeletesRewriter.dataFiles(SparkBinPackPositionDeletesRewriter.java:136)
        at 
org.apache.iceberg.spark.actions.SparkBinPackPositionDeletesRewriter.doRewrite(SparkBinPackPositionDeletesRewriter.java:106)
        at 
org.apache.iceberg.spark.actions.SparkBinPackPositionDeletesRewriter.rewrite(SparkBinPackPositionDeletesRewriter.java:79)
        at 
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.lambda$rewriteDeleteFiles$1(RewritePositionDeleteFilesSparkAction.java:195)
        at 
org.apache.iceberg.spark.actions.BaseSparkAction.withJobGroupInfo(BaseSparkAction.java:137)
        at 
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.rewriteDeleteFiles(RewritePositionDeleteFilesSparkAction.java:193)
        at 
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.lambda$doExecute$3(RewritePositionDeleteFilesSparkAction.java:238)
        at 
org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
        at org.apache.iceberg.util.Tasks$Builder.access$300(Tasks.java:69)
        at org.apache.iceberg.util.Tasks$Builder$1.run(Tasks.java:315)
   
   When I modified [this line of 
code](https://github.com/apache/iceberg/blob/master/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java#L134)
 to the following code, the problem was fixed and no error was reported.
   > Column col = col("partition.&#96;" + fields.get(i).name() + "&#96;");
   
   But delete_file always exists in the manifest:
   
![image](https://github.com/apache/iceberg/assets/5699014/e78e3a99-5072-4571-9eda-690deccbca4c)
   
   I do rewrite_position_delete_files again and it doesn't do anything.
   
![image](https://github.com/apache/iceberg/assets/5699014/b8bbf841-65f3-4c48-9edb-3ecc7f6bd35c)
   
   
   


-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to