[SPARK-33294][SQL] Add query resolved check before analyze InsertIntoDir

### What changes were proposed in this pull request?

Add `query.resolved` before analyze `InsertIntoDir`.

### Why are the changes needed?

For better error msg.
```
INSERT OVERWRITE DIRECTORY '/tmp/file' USING PARQUET
SELECT * FROM (
 SELECT c3 FROM (
  SELECT c1, c2 from values(1,2) t(c1, c2)
  )
)
```
 Before this PR, we get such error msg
```
org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to toAttribute on unresolved object, tree: *
  at org.apache.spark.sql.catalyst.analysis.Star.toAttribute(unresolved.scala:244)
  at org.apache.spark.sql.catalyst.plans.logical.Project$$anonfun$output$1.apply(basicLogicalOperators.scala:52)
  at org.apache.spark.sql.catalyst.plans.logical.Project$$anonfun$output$1.apply(basicLogicalOperators.scala:52)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.immutable.List.foreach(List.scala:392)
```

### Does this PR introduce _any_ user-facing change?

Yes, error msg changed.

### How was this patch tested?

New test.

Closes #30197 from ulysses-you/SPARK-33294.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
ulysses 2020-10-30 08:18:10 +00:00 committed by Wenchen Fan
parent 0c943cd2fb
commit d59f6a7095
2 changed files with 19 additions and 2 deletions

View file

@ -163,8 +163,8 @@ object DataSourceAnalysis extends Rule[LogicalPlan] with CastSupport {
InsertIntoDataSourceCommand(l, query, overwrite)
case InsertIntoDir(_, storage, provider, query, overwrite)
if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER =>
if query.resolved && provider.isDefined &&
provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER =>
val outputPath = new Path(storage.locationUri.get)
if (overwrite) DDLUtils.verifyNotReadPath(query, outputPath)

View file

@ -896,6 +896,23 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
sql("INSERT INTO TABLE insertTable PARTITION(part1='1', part2) SELECT 1 ,'' AS part2")
}
}
test("SPARK-33294: Add query resolved check before analyze InsertIntoDir") {
withTempPath { path =>
val msg = intercept[AnalysisException] {
sql(
s"""
|INSERT OVERWRITE DIRECTORY '${path.getAbsolutePath}' USING PARQUET
|SELECT * FROM (
| SELECT c3 FROM (
| SELECT c1, c2 from values(1,2) t(c1, c2)
| )
|)
""".stripMargin)
}.getMessage
assert(msg.contains("cannot resolve '`c3`' given input columns"))
}
}
}
class FileExistingTestFileSystem extends RawLocalFileSystem {