Skip to content

Conversation

@pan3793
Copy link
Member

@pan3793 pan3793 commented Feb 1, 2022

This is a functional regression issue in Iceberg 0.13.0. At least in Iceberg 0.12.x(I do not test every previous version), Iceberg allows creating table under the root namespace of hadoop catalog, but #3722 broke it.

Use Spark 3.2.0 and Iceberg 0.13.0, error occurs when trying to create table under root namespace of hadoop catalog.

java.sql.SQLException: Error operating EXECUTE_STATEMENT: java.lang.NegativeArraySizeException
	at java.lang.reflect.Array.newArray(Native Method)
	at java.lang.reflect.Array.newInstance(Array.java:75)
	at java.util.Arrays.copyOf(Arrays.java:3212)
	at java.util.Arrays.copyOf(Arrays.java:3181)
	at org.apache.iceberg.spark.SparkCatalog.namespaceToIdentifier(SparkCatalog.java:570)
	at org.apache.iceberg.spark.SparkCatalog.load(SparkCatalog.java:492)
	at org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:135)
	at org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:92)
	at org.apache.spark.sql.connector.catalog.TableCatalog.tableExists(TableCatalog.java:119)
	at org.apache.spark.sql.execution.datasources.v2.CreateTableExec.run(CreateTableExec.scala:40)
	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:110)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:110)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:106)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481)
	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:457)
	at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:106)
	at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:93)
	at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:91)
	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:775)
	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:618)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:613)

@pan3793
Copy link
Member Author

pan3793 commented Feb 2, 2022

cc @wypoon @rdblue

}

private Identifier namespaceToIdentifier(String[] namespace) {
assert namespace.length > 0;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't use assertions. If this is worth checking, then use a Precondition to create a readable error message.


public TestCreateTable(String catalogName, String implementation, Map<String, String> config) {
super(catalogName, implementation, config);
this.isHadoopCatalog = "testhadoop".equals(catalogName);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's no need for a field. Can you just move this test into the Assume line?

@rdblue
Copy link
Contributor

rdblue commented Feb 2, 2022

There are a couple minor things to fix, but overall good catch. Thanks, @pan3793!

@rdblue rdblue added this to the Iceberg 0.13.1 Release milestone Feb 2, 2022
@rdblue
Copy link
Contributor

rdblue commented Feb 2, 2022

I'm adding this to 0.13.1 since it is a regression.

@pan3793
Copy link
Member Author

pan3793 commented Feb 2, 2022

Addressed comments, also ported to spark 3.0/3.1

@wypoon
Copy link
Contributor

wypoon commented Feb 2, 2022

LGTM. Thanks for catching this @pan3793!

@rdblue rdblue merged commit b3da548 into apache:master Feb 2, 2022
@rdblue
Copy link
Contributor

rdblue commented Feb 2, 2022

Thanks, @pan3793!

@pan3793 pan3793 deleted the root-ns branch February 3, 2022 12:01
amogh-jahagirdar pushed a commit to amogh-jahagirdar/iceberg that referenced this pull request Feb 10, 2022
pan3793 added a commit to pan3793/iceberg that referenced this pull request Feb 15, 2022
samarthjain pushed a commit to samarthjain/incubator-iceberg that referenced this pull request Apr 6, 2022
vanliu-tx pushed a commit to BKBASE-Plugin/iceberg that referenced this pull request May 11, 2022
sunchao pushed a commit to sunchao/iceberg that referenced this pull request May 9, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants