Fokko commented on code in PR #7910: URL: https://github.com/apache/iceberg/pull/7910#discussion_r1313113936
########## spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SetMissingRelation.scala: ########## @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.streaming.sources.WriteToMicroBatchDataSource + +/** + * When running structured streaming using `.start()` instead of `.toTable(...)`, + * the relation is empty, and the catalog functions will be missing. + * The catalog functions are required when a table is partitioned with a transform. + */ +case class SetMissingRelation(spark: SparkSession) extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan match { + case p: WriteToMicroBatchDataSource if p.relation.isEmpty => + import spark.sessionState.analyzer.CatalogAndIdentifier + val originalMultipartIdentifier = spark.sessionState.sqlParser + .parseMultipartIdentifier(p.table.name()) Review Comment: I see your concern, but as you can see in the test it works quite well. In the smoke test it passes for `table`, `catalog.schema.table`, and with `s3://bucket/wh/path`. Fixing it in Spark could also work, but then I need more pointers on where to start. I looked into the comment on the issue, but it wasn't directly obvious to me. I think we should fix this, looking at how many people are bumping into this. -- 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]
