Skip to content
Closed
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1086,20 +1086,20 @@ struct<plan:string>
+- 'UnresolvedRelation [explain_temp4], [], false

== Analyzed Logical Plan ==
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@x, [key, val]
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we simply override toString in FileIndex? One idea is to produce className(rootPaths.mString(","))

Copy link
Contributor Author

Choose a reason for hiding this comment

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

-InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
+InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]

After override toString in FileIndex, Append, spark_catalog.default.explain_temp5, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218 disappeared from result

Copy link
Member

Choose a reason for hiding this comment

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

So, it means we cannot do that. Did I understand correctly? Given that, I'm +1 with the AS-IS PR.

Copy link
Contributor Author

@LuciferYang LuciferYang Jan 16, 2023

Choose a reason for hiding this comment

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

@cloud-fan className(rootPaths.mString(",")) will print out the path related to the running environment, like

org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:/Users/${userNaame}/spark-source/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/explain_temp5)

It seems not friendly to developers

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So, it means we cannot do that. Did I understand correctly? Given that, I'm +1 with the AS-IS PR.

Only override the toString method cannot achieve the goal, and as I said above, rootPaths.mString(",") needs further cleaning to avoid differences in test execution paths, so I prefer the current way.

+- Project [key#x, val#x]
+- SubqueryAlias spark_catalog.default.explain_temp4
+- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet

== Optimized Logical Plan ==
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@x, [key, val]
+- WriteFiles
+- Sort [val#x ASC NULLS FIRST], false
+- Project [key#x, empty2null(val#x) AS val#x]
+- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet

== Physical Plan ==
Execute InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
Execute InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@x, [key, val]
+- WriteFiles
+- *Sort [val#x ASC NULLS FIRST], false, 0
+- *Project [key#x, empty2null(val#x) AS val#x]
Expand Down
6 changes: 3 additions & 3 deletions sql/core/src/test/resources/sql-tests/results/explain.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -1028,20 +1028,20 @@ struct<plan:string>
+- 'UnresolvedRelation [explain_temp4], [], false

== Analyzed Logical Plan ==
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@x, [key, val]
+- Project [key#x, val#x]
+- SubqueryAlias spark_catalog.default.explain_temp4
+- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet

== Optimized Logical Plan ==
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@x, [key, val]
+- WriteFiles
+- Sort [val#x ASC NULLS FIRST], false
+- Project [key#x, empty2null(val#x) AS val#x]
+- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet

== Physical Plan ==
Execute InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@7d811218, [key, val]
Execute InsertIntoHadoopFsRelationCommand Location [not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex@x, [key, val]
+- WriteFiles
+- *Sort [val#x ASC NULLS FIRST], false, 0
+- *Project [key#x, empty2null(val#x) AS val#x]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ trait SQLQueryTestHelper {
.replaceAll("Last Access.*", s"Last Access $notIncludedMsg")
.replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg")
.replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds
.replaceAll("@[0-9a-z]+,", "@x,") // remove hashCode
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor Author

@LuciferYang LuciferYang Jan 16, 2023

Choose a reason for hiding this comment

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

@cloud-fan @dongjoon-hyun @ulysses-you

Change to add a new replaceAll action to SQLQueryTestHelper#replaceNotIncludedMsg to replace @hashCode to @x.

Copy link
Member

Choose a reason for hiding this comment

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

or maybe just remove all? since @x is uesless anyway.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

this one change to remove all

}


Expand Down