Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[branch-2.3][SPARK-25313][SQL] Fix regression in FileFormatWriter output names #22346

Closed

Conversation

gengliangwang
Copy link
Member

@gengliangwang gengliangwang commented Sep 6, 2018

Port #22320 to branch-2.3

What changes were proposed in this pull request?

Let's see the follow example:

        val location = "/tmp/t"
        val df = spark.range(10).toDF("id")
        df.write.format("parquet").saveAsTable("tbl")
        spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl")
        spark.sql(s"CREATE TABLE tbl2(ID long) USING parquet location $location")
        spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1")
        println(spark.read.parquet(location).schema)
        spark.table("tbl2").show()

The output column name in schema will be id instead of ID, thus the last query shows nothing from tbl2.
By enabling the debug message we can see that the output naming is changed from ID to id, and then the outputColumns in InsertIntoHadoopFsRelationCommand is changed in RemoveRedundantAliases.
wechatimg5

wechatimg4

To guarantee correctness, we should change the output columns from Seq[Attribute] to Seq[String] to avoid its names being replaced by optimizer.

I will fix project elimination related rules in #22311 after this one.

How was this patch tested?

Unit test.

## What changes were proposed in this pull request?

Let's see the follow example:
```
        val location = "/tmp/t"
        val df = spark.range(10).toDF("id")
        df.write.format("parquet").saveAsTable("tbl")
        spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl")
        spark.sql(s"CREATE TABLE tbl2(ID long) USING parquet location $location")
        spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1")
        println(spark.read.parquet(location).schema)
        spark.table("tbl2").show()
```
The output column name in schema will be `id` instead of `ID`, thus the last query shows nothing from `tbl2`.
By enabling the debug message we can see that the output naming is changed from `ID` to `id`, and then the `outputColumns` in `InsertIntoHadoopFsRelationCommand` is changed in `RemoveRedundantAliases`.
![wechatimg5](https://user-images.githubusercontent.com/1097932/44947871-6299f200-ae46-11e8-9c96-d45fe368206c.jpeg)

![wechatimg4](https://user-images.githubusercontent.com/1097932/44947866-56ae3000-ae46-11e8-8923-8b3bbe060075.jpeg)

**To guarantee correctness**, we should change the output columns from `Seq[Attribute]` to `Seq[String]` to avoid its names being replaced by optimizer.

I will fix project elimination related rules in apache#22311 after this one.

## How was this patch tested?

Unit test.

Closes apache#22320 from gengliangwang/fixOutputSchema.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
@SparkQA
Copy link

SparkQA commented Sep 6, 2018

Test build #95739 has finished for PR 22346 at commit 470efdc.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Sep 6, 2018

Test build #95743 has finished for PR 22346 at commit 470efdc.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to 2.3!

asfgit pushed a commit that referenced this pull request Sep 6, 2018
…put names

Port #22320 to branch-2.3
## What changes were proposed in this pull request?

Let's see the follow example:
```
        val location = "/tmp/t"
        val df = spark.range(10).toDF("id")
        df.write.format("parquet").saveAsTable("tbl")
        spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl")
        spark.sql(s"CREATE TABLE tbl2(ID long) USING parquet location $location")
        spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1")
        println(spark.read.parquet(location).schema)
        spark.table("tbl2").show()
```
The output column name in schema will be `id` instead of `ID`, thus the last query shows nothing from `tbl2`.
By enabling the debug message we can see that the output naming is changed from `ID` to `id`, and then the `outputColumns` in `InsertIntoHadoopFsRelationCommand` is changed in `RemoveRedundantAliases`.
![wechatimg5](https://user-images.githubusercontent.com/1097932/44947871-6299f200-ae46-11e8-9c96-d45fe368206c.jpeg)

![wechatimg4](https://user-images.githubusercontent.com/1097932/44947866-56ae3000-ae46-11e8-8923-8b3bbe060075.jpeg)

**To guarantee correctness**, we should change the output columns from `Seq[Attribute]` to `Seq[String]` to avoid its names being replaced by optimizer.

I will fix project elimination related rules in #22311 after this one.

## How was this patch tested?

Unit test.

Closes #22346 from gengliangwang/portSchemaOutputName2.3.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
@gengliangwang
Copy link
Member Author

Close this PR since it is merged.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants