Skip to content

Commit b78b776

Browse files
Ngone51HyukjinKwon
authored andcommitted
[SPARK-32466][SQL][FOLLOW-UP] Normalize Location info in explain plan
### What changes were proposed in this pull request? 1. Extract `SQLQueryTestSuite.replaceNotIncludedMsg` to `PlanTest`. 2. Reuse `replaceNotIncludedMsg` to normalize the explain plan that generated in `PlanStabilitySuite`. ### Why are the changes needed? This's a follow-up of #29270. Eliminates the personal related information (e.g., local directories) in the explain plan. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Updated test. Closes #29537 from Ngone51/follow-up-plan-stablity. Authored-by: yi.wu <[email protected]> Signed-off-by: HyukjinKwon <[email protected]>
1 parent c26a976 commit b78b776

File tree

3 files changed

+34
-27
lines changed

3 files changed

+34
-27
lines changed

sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -91,7 +91,7 @@ TakeOrderedAndProject (87)
9191
(1) Scan parquet default.store_sales
9292
Output [4]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_ext_discount_amt#3, ss_ext_list_price#4]
9393
Batched: true
94-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales]
94+
Location [not included in comparison]/{warehouse_dir}/store_sales]
9595
PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)]
9696
ReadSchema: struct<ss_sold_date_sk:int,ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)>
9797

@@ -105,7 +105,7 @@ Condition : (isnotnull(ss_customer_sk#2) AND isnotnull(ss_sold_date_sk#1))
105105
(4) Scan parquet default.date_dim
106106
Output [2]: [d_date_sk#5, d_year#6]
107107
Batched: true
108-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim]
108+
Location [not included in comparison]/{warehouse_dir}/date_dim]
109109
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
110110
ReadSchema: struct<d_date_sk:int,d_year:int>
111111

@@ -140,7 +140,7 @@ Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0
140140
(12) Scan parquet default.customer
141141
Output [8]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12, c_preferred_cust_flag#13, c_birth_country#14, c_login#15, c_email_address#16]
142142
Batched: true
143-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/customer]
143+
Location [not included in comparison]/{warehouse_dir}/customer]
144144
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
145145
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string>
146146

@@ -201,7 +201,7 @@ Arguments: [customer_id#22 ASC NULLS FIRST], false, 0
201201
(25) Scan parquet default.store_sales
202202
Output [4]: [ss_sold_date_sk#1, ss_customer_sk#2, ss_ext_discount_amt#3, ss_ext_list_price#4]
203203
Batched: true
204-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/store_sales]
204+
Location [not included in comparison]/{warehouse_dir}/store_sales]
205205
PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)]
206206
ReadSchema: struct<ss_sold_date_sk:int,ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)>
207207

@@ -215,7 +215,7 @@ Condition : (isnotnull(ss_customer_sk#2) AND isnotnull(ss_sold_date_sk#1))
215215
(28) Scan parquet default.date_dim
216216
Output [2]: [d_date_sk#5, d_year#6]
217217
Batched: true
218-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/date_dim]
218+
Location [not included in comparison]/{warehouse_dir}/date_dim]
219219
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
220220
ReadSchema: struct<d_date_sk:int,d_year:int>
221221

@@ -301,7 +301,7 @@ Input [5]: [customer_id#22, year_total#23, customer_id#31, customer_preferred_cu
301301
(47) Scan parquet default.web_sales
302302
Output [4]: [ws_sold_date_sk#35, ws_bill_customer_sk#36, ws_ext_discount_amt#37, ws_ext_list_price#38]
303303
Batched: true
304-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales]
304+
Location [not included in comparison]/{warehouse_dir}/web_sales]
305305
PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)]
306306
ReadSchema: struct<ws_sold_date_sk:int,ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)>
307307

@@ -394,7 +394,7 @@ Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year
394394
(68) Scan parquet default.web_sales
395395
Output [4]: [ws_sold_date_sk#35, ws_bill_customer_sk#36, ws_ext_discount_amt#37, ws_ext_list_price#38]
396396
Batched: true
397-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilityWithStatsSuite/web_sales]
397+
Location [not included in comparison]/{warehouse_dir}/web_sales]
398398
PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)]
399399
ReadSchema: struct<ws_sold_date_sk:int,ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)>
400400

sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ TakeOrderedAndProject (73)
7777
(1) Scan parquet default.customer
7878
Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
7979
Batched: true
80-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer]
80+
Location [not included in comparison]/{warehouse_dir}/customer]
8181
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
8282
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string>
8383

@@ -91,7 +91,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2))
9191
(4) Scan parquet default.store_sales
9292
Output [4]: [ss_sold_date_sk#9, ss_customer_sk#10, ss_ext_discount_amt#11, ss_ext_list_price#12]
9393
Batched: true
94-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/store_sales]
94+
Location [not included in comparison]/{warehouse_dir}/store_sales]
9595
PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)]
9696
ReadSchema: struct<ss_sold_date_sk:int,ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)>
9797

@@ -118,7 +118,7 @@ Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_
118118
(10) Scan parquet default.date_dim
119119
Output [2]: [d_date_sk#14, d_year#15]
120120
Batched: true
121-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim]
121+
Location [not included in comparison]/{warehouse_dir}/date_dim]
122122
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
123123
ReadSchema: struct<d_date_sk:int,d_year:int>
124124

@@ -167,7 +167,7 @@ Condition : (isnotnull(year_total#22) AND (year_total#22 > 0.00))
167167
(20) Scan parquet default.customer
168168
Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
169169
Batched: true
170-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer]
170+
Location [not included in comparison]/{warehouse_dir}/customer]
171171
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
172172
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string>
173173

@@ -193,7 +193,7 @@ Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_
193193
(26) Scan parquet default.date_dim
194194
Output [2]: [d_date_sk#14, d_year#15]
195195
Batched: true
196-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/date_dim]
196+
Location [not included in comparison]/{warehouse_dir}/date_dim]
197197
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
198198
ReadSchema: struct<d_date_sk:int,d_year:int>
199199

@@ -251,7 +251,7 @@ Input [5]: [customer_id#21, year_total#22, customer_id#28, customer_preferred_cu
251251
(38) Scan parquet default.customer
252252
Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
253253
Batched: true
254-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer]
254+
Location [not included in comparison]/{warehouse_dir}/customer]
255255
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
256256
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string>
257257

@@ -265,7 +265,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2))
265265
(41) Scan parquet default.web_sales
266266
Output [4]: [ws_sold_date_sk#32, ws_bill_customer_sk#33, ws_ext_discount_amt#34, ws_ext_list_price#35]
267267
Batched: true
268-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/web_sales]
268+
Location [not included in comparison]/{warehouse_dir}/web_sales]
269269
PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)]
270270
ReadSchema: struct<ws_sold_date_sk:int,ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)>
271271

@@ -343,7 +343,7 @@ Input [6]: [customer_id#21, year_total#22, customer_preferred_cust_flag#29, year
343343
(58) Scan parquet default.customer
344344
Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
345345
Batched: true
346-
Location: InMemoryFileIndex [file:/Users/yi.wu/IdeaProjects/spark/sql/core/spark-warehouse/org.apache.spark.sql.TPCDSV1_4_PlanStabilitySuite/customer]
346+
Location [not included in comparison]/{warehouse_dir}/customer]
347347
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
348348
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string>
349349

sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala

Lines changed: 19 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -94,6 +94,8 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite {
9494
private val referenceRegex = "#\\d+".r
9595
private val normalizeRegex = "#\\d+L?".r
9696

97+
private val clsName = this.getClass.getCanonicalName
98+
9799
def goldenFilePath: String
98100

99101
private def getDirForTest(name: String): File = {
@@ -102,8 +104,8 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite {
102104

103105
private def isApproved(dir: File, actualSimplifiedPlan: String): Boolean = {
104106
val file = new File(dir, "simplified.txt")
105-
val approved = FileUtils.readFileToString(file, StandardCharsets.UTF_8)
106-
approved == actualSimplifiedPlan
107+
val expected = FileUtils.readFileToString(file, StandardCharsets.UTF_8)
108+
expected == actualSimplifiedPlan
107109
}
108110

109111
/**
@@ -115,7 +117,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite {
115117
* @param explain the full explain output; this is saved to help debug later as the simplified
116118
* plan is not too useful for debugging
117119
*/
118-
private def generateApprovedPlanFile(plan: SparkPlan, name: String, explain: String): Unit = {
120+
private def generateGoldenFile(plan: SparkPlan, name: String, explain: String): Unit = {
119121
val dir = getDirForTest(name)
120122
val simplified = getSimplifiedPlan(plan)
121123
val foundMatch = dir.exists() && isApproved(dir, simplified)
@@ -207,7 +209,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite {
207209
* WholeStageCodegen
208210
* Project [c_customer_id]
209211
*/
210-
def getSimplifiedPlan(node: SparkPlan, depth: Int): String = {
212+
def simplifyNode(node: SparkPlan, depth: Int): String = {
211213
val padding = " " * depth
212214
var thisNode = node.nodeName
213215
if (node.references.nonEmpty) {
@@ -220,19 +222,24 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite {
220222
if (id > 0) {
221223
thisNode += s" #$id"
222224
}
223-
val childrenSimplified = node.children.map(getSimplifiedPlan(_, depth + 1))
224-
val subqueriesSimplified = node.subqueries.map(getSimplifiedPlan(_, depth + 1))
225+
val childrenSimplified = node.children.map(simplifyNode(_, depth + 1))
226+
val subqueriesSimplified = node.subqueries.map(simplifyNode(_, depth + 1))
225227
s"$padding$thisNode\n${subqueriesSimplified.mkString("")}${childrenSimplified.mkString("")}"
226228
}
227229

228-
getSimplifiedPlan(plan, 0)
230+
simplifyNode(plan, 0)
229231
}
230232

231-
private def normalizeIds(query: String): String = {
233+
private def normalizeIds(plan: String): String = {
232234
val map = new mutable.HashMap[String, String]()
233-
normalizeRegex.findAllMatchIn(query).map(_.toString)
235+
normalizeRegex.findAllMatchIn(plan).map(_.toString)
234236
.foreach(map.getOrElseUpdate(_, (map.size + 1).toString))
235-
normalizeRegex.replaceAllIn(query, regexMatch => s"#${map(regexMatch.toString)}")
237+
normalizeRegex.replaceAllIn(plan, regexMatch => s"#${map(regexMatch.toString)}")
238+
}
239+
240+
private def normalizeLocation(plan: String): String = {
241+
plan.replaceAll(s"Location.*$clsName/",
242+
"Location [not included in comparison]/{warehouse_dir}/")
236243
}
237244

238245
/**
@@ -244,10 +251,10 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite {
244251
classLoader = Thread.currentThread().getContextClassLoader)
245252
val qe = sql(queryString).queryExecution
246253
val plan = qe.executedPlan
247-
val explain = normalizeIds(qe.explainString(FormattedMode))
254+
val explain = normalizeLocation(normalizeIds(qe.explainString(FormattedMode)))
248255

249256
if (regenerateGoldenFiles) {
250-
generateApprovedPlanFile(plan, query + suffix, explain)
257+
generateGoldenFile(plan, query + suffix, explain)
251258
} else {
252259
checkWithApproved(plan, query + suffix, explain)
253260
}

0 commit comments

Comments
 (0)