holdenk commented on issue #8614:
URL: https://github.com/apache/iceberg/issues/8614#issuecomment-1732943445
Ah here we go for the repro to work it needs data:
```
DROP TABLE IF EXISTS local.udevelopers_sorted;
CREATE TABLE IF NOT EXISTS local.udevelopers_sorted (
username string,
firstname string,
lastname string)
USING ICEBERG;
INSERT INTO local.udevelopers_sorted VALUES("krisnova", "Kris", "Nova");
ALTER TABLE local.udevelopers_sorted WRITE ORDERED BY lastname;
ALTER TABLE local.udevelopers_sorted RENAME COLUMN lastname TO
deprecated_lastname;
SELECT * FROM local.udevelopers_sorted;
ALTER TABLE local.udevelopers_sorted WRITE ORDERED BY username;
ALTER TABLE local.udevelopers_sorted DROP COLUMN deprecated_lastname;
SELECT * FROM local.udevelopers_sorted;
```
Gives me the error:
```
org.apache.iceberg.exceptions.ValidationException: Cannot find source column
for sort field: identity(3) ASC NULLS FIRST
at
org.apache.iceberg.exceptions.ValidationException.check(ValidationException.java:49)
at org.apache.iceberg.SortOrder.checkCompatibility(SortOrder.java:300)
at org.apache.iceberg.SortOrder$Builder.build(SortOrder.java:264)
at org.apache.iceberg.UnboundSortOrder.bind(UnboundSortOrder.java:54)
at org.apache.iceberg.SortOrderParser.fromJson(SortOrderParser.java:116)
at
org.apache.iceberg.TableMetadataParser.fromJson(TableMetadataParser.java:429)
at
org.apache.iceberg.TableMetadataParser.fromJson(TableMetadataParser.java:309)
at
org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:274)
at
org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:267)
at
org.apache.iceberg.hadoop.HadoopTableOperations.updateVersionAndMetadata(HadoopTableOperations.java:98)
at
org.apache.iceberg.hadoop.HadoopTableOperations.refresh(HadoopTableOperations.java:121)
at
org.apache.iceberg.hadoop.HadoopTableOperations.current(HadoopTableOperations.java:84)
at
org.apache.iceberg.BaseTransaction.committedFiles(BaseTransaction.java:495)
at
org.apache.iceberg.BaseTransaction.commitSimpleTransaction(BaseTransaction.java:430)
at
org.apache.iceberg.BaseTransaction.commitTransaction(BaseTransaction.java:277)
at
org.apache.iceberg.spark.SparkCatalog.commitChanges(SparkCatalog.java:587)
at
org.apache.iceberg.spark.SparkCatalog.alterTable(SparkCatalog.java:303)
at
org.apache.spark.sql.execution.datasources.v2.AlterTableExec.run(AlterTableExec.scala:37)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:109)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:169)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
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.transformDown(TreeNode.scala:560)
at
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:94)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:81)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:79)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:219)
at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:99)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96)
at
org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:622)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:617)
at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:651)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:67)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:384)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1(SparkSQLCLIDriver.scala:504)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1$adapted(SparkSQLCLIDriver.scala:498)
at scala.collection.Iterator.foreach(Iterator.scala:943)
at scala.collection.Iterator.foreach$(Iterator.scala:943)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
at scala.collection.IterableLike.foreach(IterableLike.scala:74)
at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)
at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)
at
org.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)
at org.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)
at
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at
org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
at
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)
at
org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)
at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)
at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
at
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)
at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```
This (thankfully) does not result in the corrupted schema in Iceberg 1.3 I
saw with the full environment, but does result in failure.
--
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]