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

   ### Apache Iceberg version
   
   1.10.0 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   Hi,
   
   I think what I will describe next is a bug since I searched for info that 
could explain the behavior and did not find anything.
   
   I am running a bunch of Iceberg maintenance procedures with Spark, I 
recently migrated my jobs from Spark 3.5.6 to 4.0.1, and after this some of my 
maintenance procedure jobs that are performing a `rewrite_data_files` with a 
zOrder started to fail.
   
   The pattern I observe is that I encounter the error when one of the column 
is of type DATE.
   
   The error I get is:
   ```
   ERROR RewriteDataFilesSparkAction: Cannot complete rewrite, 
partial-progress.enabled is not enabled and one of the file set groups failed 
to be rewritten. This error occurred during the writing of new files, not 
during the commit process. This indicates something is wrong that doesn't 
involve conflicts with other Iceberg operations. Enabling 
partial-progress.enabled may help in this case but the root cause should be 
investigated. Cleaning up 0 groups which finished being written.
   java.lang.RuntimeException: org.apache.spark.sql.AnalysisException: 
[DATATYPE_MISMATCH.CAST_WITH_FUNC_SUGGESTION] Cannot resolve "CAST(trade_date 
AS BIGINT)" due to data type mismatch: cannot cast "DATE" to "BIGINT".
   To convert values from "DATE" to "BIGINT", you can use the functions 
`UNIX_DATE` instead. SQLSTATE: 42K09;
   'Project [trade_date#160, sec_id#161, ticker#162, adjustment_factor#163, 
cumulative_factor_price#164, cumulative_factor_volume#165, 
adjustment_reason#166, event_type#167, 
INTERLEAVE_BYTES('array(STRING-LEXICAL-BYTES(ticker#162), 
INT_ORDERED_BYTES(sec_id#161), LONG_ORDERED_BYTES(cast(trade_date#160 as 
bigint)))) AS ICEZVALUE#173]
   +- RelationV2[trade_date#160, sec_id#161, ticker#162, adjustment_factor#163, 
cumulative_factor_price#164, cumulative_factor_volume#165, 
adjustment_reason#166, event_type#167] 
default_cache_iceberg.`db876375-7104-4e5e-be50-5364aabc9a09#rewrite` 
default_cache_iceberg.`db876375-7104-4e5e-be50-5364aabc9a09#rewrite`
   ```
   
   I run the following query:
   ```sql
   spark.sql("""--sql
   CALL biglakeCatalog.system.rewrite_data_files(
       table => 'biglakeCatalog.silver.cumu_adj_factors_daily',
       strategy => 'sort', sort_order => 'zorder(ticker,sec_id,trade_date)',
       options => map('rewrite-all', 'true', 'target-file-size-bytes', 
'536870912', 'max-concurrent-file-group-rewrites', '5')
   );
   """).show()
   ```
   
   Where the `trade_date` column is of type DATE.
   
   All my tables are hosted in a BigLake REST Iceberg catalog in GCP, and the 
version of all my tables is `'format-version' = '2',`. To be specific, all my 
tables have the following Iceberg properties:
   ```
   ...
   TBLPROPERTIES (
       'format-version' = '2',
       'write.upsert.enabled' = 'true',
       'write.merge.mode' = 'copy-on-write',
       'commit.retry.num-retries' = '10',
       'commit.retry.min-wait-ms' = '1000',
       'write.merge.isolation-level' = 'snapshot',
       'write.metadata.delete-after-commit.enabled' = 'true',
       'history.expire.min-snapshots-to-keep' = '3',
       'history.expire.max-snapshot-age-ms' = '86400000',
       'write.wap.enabled' = 'true'
   )
   ```
   Some of the table are partitioned, but the one I use in this example is not.
   
   
   To give more context, I create my SparkSession as follows:
   
   ```python
   import pandas as pd
   from pyspark.sql import SparkSession
   
   pd.set_option("display.max_columns", None)
   pd.set_option("display.max_rows", None)
   
   catalog_name = "biglakeCatalog"
   
   spark: SparkSession = (
       SparkSession.builder.appName("Richfox Data Loader")
       .master("local[12]")
       .config("spark.driver.memory", "18g")
       .config("spark.jars.ivy", "/tmp/.ivy_spark")
       .config(
           "spark.jars",
           
"https://repo1.maven.org/maven2/org/postgresql/postgresql/42.7.7/postgresql-42.7.7.jar,";
           
"https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-4.0_2.13/1.10.0/iceberg-spark-runtime-4.0_2.13-1.10.0.jar,";
           
"https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-gcp-bundle/1.10.0/iceberg-gcp-bundle-1.10.0.jar,";
           
"https://repo1.maven.org/maven2/com/google/cloud/bigdataoss/gcs-connector/3.1.7/gcs-connector-3.1.7-shaded.jar,";
           
"https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-common/3.3.6/hadoop-common-3.3.6.jar";,
       )
       .config("spark.sql.execution.arrow.pyspark.enabled", "true")
       .config(f"spark.sql.catalog.{catalog_name}", 
"org.apache.iceberg.spark.SparkCatalog")
       .config(f"spark.sql.catalog.{catalog_name}.type", "rest")
       .config(f"spark.sql.catalog.{catalog_name}.uri", 
"https://biglake.googleapis.com/iceberg/v1beta/restcatalog";)
       .config(f"spark.sql.catalog.{catalog_name}.warehouse", "gs://some 
bucket")
       .config(f"spark.sql.catalog.{catalog_name}.header.x-goog-user-project", 
"some project")
       .config(f"spark.sql.catalog.{catalog_name}.rest.auth.type", 
"org.apache.iceberg.gcp.auth.GoogleAuthManager")
       .config(f"spark.sql.catalog.{catalog_name}.io-impl", 
"org.apache.iceberg.gcp.gcs.GCSFileIO")
       
.config(f"spark.sql.catalog.{catalog_name}.rest-metrics-reporting-enabled", 
"false")
       .config("spark.hadoop.fs.gs.project.id", "rcim-prod-data-core-0")
       .config("spark.hadoop.fs.gs.impl", 
"com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem")
       .config("spark.hadoop.google.cloud.auth.service.account.enable", "true")
       .config("spark.hadoop.fs.gs.auth.type", "APPLICATION_DEFAULT")
       .config(
           "spark.sql.extensions",
           "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions",
       )
       .getOrCreate()
   )
   ```
   
   Some remarks:
   - The same maintenance procedure is working fine when using Spark 3.5.6 
(with Iceberg 1.10.0). I use Spark 3.5.6 with Java 17 and Scala 2.12 while 
Spark 4.0.1 uses Java 21 and Scala 2.13. 
   - Some of my maintenance procedure queries do not involve DATE columns, and 
they work fine with Spark 4.0.1 (with Iceberg 1.10.0).
   
   Here is a more verbose snippet of the stacktrace:
   
   ```
   25/09/17 09:50:31 WARN RewriteDataFilesSparkAction: Failure during rewrite 
process for group FileGroupInfo{globalIndex=1, partitionIndex=1, 
partition=PartitionData{}}
   org.apache.spark.sql.AnalysisException: 
[DATATYPE_MISMATCH.CAST_WITH_FUNC_SUGGESTION] Cannot resolve "CAST(trade_date 
AS BIGINT)" due to data type mismatch: cannot cast "DATE" to "BIGINT".
   To convert values from "DATE" to "BIGINT", you can use the functions 
`UNIX_DATE` instead. SQLSTATE: 42K09;
   'Project [trade_date#160, sec_id#161, ticker#162, adjustment_factor#163, 
cumulative_factor_price#164, cumulative_factor_volume#165, 
adjustment_reason#166, event_type#167, 
INTERLEAVE_BYTES('array(STRING-LEXICAL-BYTES(ticker#162), 
INT_ORDERED_BYTES(sec_id#161), LONG_ORDERED_BYTES(cast(trade_date#160 as 
bigint)))) AS ICEZVALUE#173]
   +- RelationV2[trade_date#160, sec_id#161, ticker#162, adjustment_factor#163, 
cumulative_factor_price#164, cumulative_factor_volume#165, 
adjustment_reason#166, event_type#167] 
default_cache_iceberg.`db876375-7104-4e5e-be50-5364aabc9a09#rewrite` 
default_cache_iceberg.`db876375-7104-4e5e-be50-5364aabc9a09#rewrite`
   
        at 
org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.dataTypeMismatch(package.scala:77)
        at 
org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.dataTypeMismatch(package.scala:70)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$7(CheckAnalysis.scala:420)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$7$adapted(CheckAnalysis.scala:402)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:252)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
        at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
        at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
        at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
        at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
        at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
        at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
        at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$6(CheckAnalysis.scala:402)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$6$adapted(CheckAnalysis.scala:402)
        at scala.collection.immutable.List.foreach(List.scala:334)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$2(CheckAnalysis.scala:402)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$2$adapted(CheckAnalysis.scala:284)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:252)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis0(CheckAnalysis.scala:284)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis0$(CheckAnalysis.scala:255)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis0(Analyzer.scala:299)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis(CheckAnalysis.scala:244)
        at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis$(CheckAnalysis.scala:231)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:299)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.$anonfun$resolveInFixedPoint$1(HybridAnalyzer.scala:192)
        at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.resolveInFixedPoint(HybridAnalyzer.scala:192)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.$anonfun$apply$1(HybridAnalyzer.scala:76)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.withTrackedAnalyzerBridgeState(HybridAnalyzer.scala:111)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.apply(HybridAnalyzer.scala:71)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:330)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:423)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:330)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$2(QueryExecution.scala:110)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:148)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:278)
        at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:654)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:278)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
        at 
org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:277)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$1(QueryExecution.scala:110)
        at scala.util.Try$.apply(Try.scala:217)
        at 
org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1378)
        at 
org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1439)
        at org.apache.spark.util.LazyTry.get(LazyTry.scala:58)
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:121)
        at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:80)
        at 
org.apache.spark.sql.classic.Dataset$.$anonfun$ofRows$1(Dataset.scala:115)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
        at org.apache.spark.sql.classic.Dataset$.ofRows(Dataset.scala:113)
        at org.apache.spark.sql.classic.Dataset.withPlan(Dataset.scala:2263)
        at org.apache.spark.sql.classic.Dataset.withColumns(Dataset.scala:1283)
        at org.apache.spark.sql.classic.Dataset.withColumns(Dataset.scala:232)
        at org.apache.spark.sql.Dataset.withColumn(Dataset.scala:2187)
        at org.apache.spark.sql.classic.Dataset.withColumn(Dataset.scala:1819)
        at org.apache.spark.sql.classic.Dataset.withColumn(Dataset.scala:232)
        at 
org.apache.iceberg.spark.actions.SparkZOrderFileRewriteRunner.sortedDF(SparkZOrderFileRewriteRunner.java:128)
        at 
org.apache.iceberg.spark.actions.SparkShufflingFileRewriteRunner.doRewrite(SparkShufflingFileRewriteRunner.java:115)
        at 
org.apache.iceberg.spark.actions.SparkDataFileRewriteRunner.rewrite(SparkDataFileRewriteRunner.java:52)
        at 
org.apache.iceberg.spark.actions.SparkDataFileRewriteRunner.rewrite(SparkDataFileRewriteRunner.java:33)
        at 
org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.lambda$rewriteFiles$0(RewriteDataFilesSparkAction.java:215)
        at 
org.apache.iceberg.spark.JobGroupUtils.withJobGroupInfo(JobGroupUtils.java:59)
        at 
org.apache.iceberg.spark.JobGroupUtils.withJobGroupInfo(JobGroupUtils.java:51)
        at 
org.apache.iceberg.spark.actions.BaseSparkAction.withJobGroupInfo(BaseSparkAction.java:130)
        at 
org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.rewriteFiles(RewriteDataFilesSparkAction.java:214)
        at 
org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.lambda$doExecute$2(RewriteDataFilesSparkAction.java:257)
        at 
org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
        at org.apache.iceberg.util.Tasks$Builder$1.run(Tasks.java:315)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
        at java.base/java.lang.Thread.run(Thread.java:1583)
        Suppressed: org.apache.spark.util.Utils$OriginalTryStackTraceException: 
Full stacktrace of original doTryWithCallerStacktrace caller
                at 
org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.dataTypeMismatch(package.scala:77)
                at 
org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.dataTypeMismatch(package.scala:70)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$7(CheckAnalysis.scala:420)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$7$adapted(CheckAnalysis.scala:402)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:252)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
                at 
scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
                at 
scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
                at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
                at 
scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
                at 
scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
                at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
                at 
scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619)
                at 
scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617)
                at scala.collection.AbstractIterable.foreach(Iterable.scala:935)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:251)
                at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:251)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$6(CheckAnalysis.scala:402)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$6$adapted(CheckAnalysis.scala:402)
                at scala.collection.immutable.List.foreach(List.scala:334)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$2(CheckAnalysis.scala:402)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$2$adapted(CheckAnalysis.scala:284)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:252)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis0(CheckAnalysis.scala:284)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis0$(CheckAnalysis.scala:255)
                at 
org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis0(Analyzer.scala:299)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis(CheckAnalysis.scala:244)
                at 
org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis$(CheckAnalysis.scala:231)
                at 
org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:299)
                at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.$anonfun$resolveInFixedPoint$1(HybridAnalyzer.scala:192)
                at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
                at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89)
                at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.resolveInFixedPoint(HybridAnalyzer.scala:192)
                at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.$anonfun$apply$1(HybridAnalyzer.scala:76)
                at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.withTrackedAnalyzerBridgeState(HybridAnalyzer.scala:111)
                at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.apply(HybridAnalyzer.scala:71)
                at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:330)
                at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:423)
                at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:330)
                at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$2(QueryExecution.scala:110)
                at 
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:148)
                at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:278)
                at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:654)
                at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:278)
                at 
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
                at 
org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:277)
                at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$1(QueryExecution.scala:110)
                at scala.util.Try$.apply(Try.scala:217)
                at 
org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1378)
                at 
org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46)
                at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46)
                ... 29 more
   25/09/17 09:50:31 ERROR RewriteDataFilesSparkAction: Cannot complete 
rewrite, partial-progress.enabled is not enabled and one of the file set groups 
failed to be rewritten. This error occurred during the writing of new files, 
not during the commit process. This indicates something is wrong that doesn't 
involve conflicts with other Iceberg operations. Enabling 
partial-progress.enabled may help in this case but the root cause should be 
investigated. Cleaning up 0 groups which finished being written.
   java.lang.RuntimeException: org.apache.spark.sql.AnalysisException: 
[DATATYPE_MISMATCH.CAST_WITH_FUNC_SUGGESTION] Cannot resolve "CAST(trade_date 
AS BIGINT)" due to data type mismatch: cannot cast "DATE" to "BIGINT".
   To convert values from "DATE" to "BIGINT", you can use the functions 
`UNIX_DATE` instead. SQLSTATE: 42K09;
   'Project [trade_date#160, sec_id#161, ticker#162, adjustment_factor#163, 
cumulative_factor_price#164, cumulative_factor_volume#165, 
adjustment_reason#166, event_type#167, 
INTERLEAVE_BYTES('array(STRING-LEXICAL-BYTES(ticker#162), 
INT_ORDERED_BYTES(sec_id#161), LONG_ORDERED_BYTES(cast(trade_date#160 as 
bigint)))) AS ICEZVALUE#173]
   +- RelationV2[trade_date#160, sec_id#161, ticker#162, adjustment_factor#163, 
cumulative_factor_price#164, cumulative_factor_volume#165, 
adjustment_reason#166, event_type#167] 
default_cache_iceberg.`db876375-7104-4e5e-be50-5364aabc9a09#rewrite` 
default_cache_iceberg.`db876375-7104-4e5e-be50-5364aabc9a09#rewrite`
   
        at 
org.apache.iceberg.util.ExceptionUtil.castAndThrow(ExceptionUtil.java:39)
        at org.apache.iceberg.util.Tasks.throwOne(Tasks.java:599)
        at org.apache.iceberg.util.Tasks$Builder.runParallel(Tasks.java:394)
        at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:201)
        at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
        at 
org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.doExecute(RewriteDataFilesSparkAction.java:255)
        at 
org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.execute(RewriteDataFilesSparkAction.java:180)
        at 
org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.execute(RewriteDataFilesSparkAction.java:68)
        at 
org.apache.iceberg.spark.procedures.RewriteDataFilesProcedure.lambda$call$0(RewriteDataFilesProcedure.java:129)
        at 
org.apache.iceberg.spark.procedures.BaseProcedure.execute(BaseProcedure.java:132)
        at 
org.apache.iceberg.spark.procedures.BaseProcedure.modifyIcebergTable(BaseProcedure.java:113)
        at 
org.apache.iceberg.spark.procedures.RewriteDataFilesProcedure.call(RewriteDataFilesProcedure.java:118)
        at 
org.apache.spark.sql.catalyst.analysis.InvokeProcedures.org$apache$spark$sql$catalyst$analysis$InvokeProcedures$$invoke(InvokeProcedures.scala:48)
        at 
org.apache.spark.sql.catalyst.analysis.InvokeProcedures$$anonfun$apply$1.applyOrElse(InvokeProcedures.scala:40)
        at 
org.apache.spark.sql.catalyst.analysis.InvokeProcedures$$anonfun$apply$1.applyOrElse(InvokeProcedures.scala:36)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDownWithPruning$2(AnalysisHelper.scala:200)
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDownWithPruning$1(AnalysisHelper.scala:200)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:416)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDownWithPruning(AnalysisHelper.scala:198)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDownWithPruning$(AnalysisHelper.scala:194)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDownWithPruning(LogicalPlan.scala:37)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsWithPruning(AnalysisHelper.scala:100)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsWithPruning$(AnalysisHelper.scala:97)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsWithPruning(LogicalPlan.scala:37)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:77)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:76)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:37)
        at 
org.apache.spark.sql.catalyst.analysis.InvokeProcedures.apply(InvokeProcedures.scala:36)
        at 
org.apache.spark.sql.catalyst.analysis.InvokeProcedures.apply(InvokeProcedures.scala:34)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242)
        at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183)
        at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179)
        at scala.collection.immutable.List.foldLeft(List.scala:79)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231)
        at scala.collection.immutable.List.foreach(List.scala:334)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:340)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:336)
        at 
org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:234)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:336)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:299)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.resolveInFixedPoint(HybridAnalyzer.scala:190)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.$anonfun$apply$1(HybridAnalyzer.scala:76)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.withTrackedAnalyzerBridgeState(HybridAnalyzer.scala:111)
        at 
org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.apply(HybridAnalyzer.scala:71)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:330)
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:423)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:330)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$2(QueryExecution.scala:110)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:148)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:278)
        at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:654)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:278)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
        at 
org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:277)
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$1(QueryExecution.scala:110)
        at scala.util.Try$.apply(Try.scala:217)
        at 
org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1378)
        at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46)
        at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46)
        at org.apache.spark.util.LazyTry.get(LazyTry.scala:58)
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:121)
        at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:80)
        at 
org.apache.spark.sql.classic.Dataset$.$anonfun$ofRows$5(Dataset.scala:139)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
        at org.apache.spark.sql.classic.Dataset$.ofRows(Dataset.scala:136)
        at 
org.apache.spark.sql.classic.SparkSession.$anonfun$sql$1(SparkSession.scala:462)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
        at org.apache.spark.sql.classic.SparkSession.sql(SparkSession.scala:449)
        at org.apache.spark.sql.classic.SparkSession.sql(SparkSession.scala:467)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:75)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:52)
        at java.base/java.lang.reflect.Method.invoke(Method.java:580)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:374)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at 
py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:184)
        at py4j.ClientServerConnection.run(ClientServerConnection.java:108)
        at java.base/java.lang.Thread.run(Thread.java:1583)
   ```
   
   ### Willingness to contribute
   
   - [ ] I can contribute a fix for this bug independently
   - [ ] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [x] I cannot contribute a fix for this bug at this time


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