You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "dtenedor (via GitHub)" <gi...@apache.org> on 2023/03/15 18:19:17 UTC

[GitHub] [spark] dtenedor opened a new pull request, #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

dtenedor opened a new pull request, #40449:
URL: https://github.com/apache/spark/pull/40449

   ### What changes were proposed in this pull request?
   
   This PR creates a new `SQLAnalyzerTestSuite` that consumes input SQL queries from files and then performs analysis and generates the string representation of the analyzed plans. It works much the same way as the existing `SQLQueryTestSuite` except that it produces analyzed plan string instead of query results in the output golden files.
   
   ### Why are the changes needed?
   
   This framework will help us guard against bugs in future development by showing a clear signal when analyzer updates result in changes to the query plans for a body of test queries. PR authors and reviewers will be able to see the diffs for all changed plans during the review, and any unintentional plan changes will act as a signal for PR authors to adjust their code to prevent the changes from happening.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   This test adds a new test suite and initial golden file. The new test suite passes as of this PR.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1470801546

   @dtenedor Since we already have `SQLQueryTestSuite` which has good basic Spark SQL features coverage, shall we combine both? E.g. let `SQLQueryTestSuite` show analyzed plan/optimized plan/execution results for comparison, and port more tests from [zetasql](https://github.com/google/zetasql/tree/master/zetasql/analyzer/testdata)
   What is the reason for having a new analysis test only?


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139386812


##########
sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:
##########
@@ -228,6 +228,7 @@ abstract class QueryTest extends PlanTest {
     assert(query.queryExecution.executedPlan.missingInput.isEmpty,
       s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
   }
+

Review Comment:
   Sounds good, done.



##########
sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out:
##########


Review Comment:
   Yes, just uploaded 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1472785577

   LGTM except for minor comments. Thanks for the work!


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1476454408

   https://github.com/apache/spark/pull/40492


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139827725


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -678,4 +663,99 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       super.afterAll()
     }
   }
+
+  /**
+   * Consumes contents from a single golden file and compares the expected results against the
+   * output of running a query.
+   */
+  def readGoldenFileAndCompareResults(
+    resultFile: String,
+    outputs: Seq[QueryTestOutput],
+    makeOutput: (String, Option[String], String) => QueryTestOutput): Unit = {
+    // Read back the golden file.
+    val expectedOutputs: Seq[QueryTestOutput] = {
+      val goldenOutput = fileToString(new File(resultFile))
+      val segments = goldenOutput.split("-- !query.*\n")
+
+      val numSegments = outputs.map(_.numSegments).sum + 1
+      assert(segments.size == numSegments,
+        s"Expected $numSegments blocks in result file but got " +
+          s"${segments.size}. Try regenerate the result files.")
+      var curSegment = 0
+      outputs.map { output =>
+        val result = if (output.numSegments == 3) {
+          makeOutput(
+            segments(curSegment + 1).trim, // SQL
+            Some(segments(curSegment + 2).trim), // Schema
+            segments(curSegment + 3).replaceAll("\\s+$", "")) // Output
+        } else {
+          makeOutput(
+            segments(curSegment + 1).trim, // SQL
+            None, // Schema
+            segments(curSegment + 2).replaceAll("\\s+$", "")) // Output
+        }
+        curSegment += output.numSegments
+        result
+      }
+    }
+
+    // Compare results.
+    assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") {
+      outputs.size
+    }
+
+    outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) =>
+      assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") {
+        output.sql
+      }
+      assertResult(expected.schema,
+        s"Schema did not match for query #$i\n${expected.sql}: $output") {
+        output.schema
+      }
+      assertResult(expected.output, s"Result did not match" +
+        s" for query #$i\n${expected.sql}") {
+        output.output
+      }
+    }
+  }
+
+  /** A single SQL query's output. */
+  trait QueryTestOutput {
+    def sql: String
+    def schema: Option[String]
+    def output: String
+    def numSegments: Int
+  }
+
+  /** A single SQL query's output. */

Review Comment:
   ```suggestion
     /** A single SQL query's execution output. */
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139832774


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -460,49 +473,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
     }
 
     withClue(clue) {
-      // Read back the golden file.
-      val expectedOutputs: Seq[QueryOutput] = {
-        val goldenOutput = fileToString(new File(testCase.resultFile))
-        val segments = goldenOutput.split("-- !query.*\n")
-
-        // each query has 3 segments, plus the header
-        assert(segments.size == outputs.size * 3 + 1,
-          s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " +
-            s"Try regenerate the result files.")
-        Seq.tabulate(outputs.size) { i =>
-          QueryOutput(
-            sql = segments(i * 3 + 1).trim,
-            schema = segments(i * 3 + 2).trim,
-            output = segments(i * 3 + 3).replaceAll("\\s+$", "")
-          )
-        }
-      }
-
-      // Compare results.
-      assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") {
-        outputs.size
-      }
-
-      outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) =>
-        assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") {
-          output.sql
-        }
-        assertResult(expected.schema,
-          s"Schema did not match for query #$i\n${expected.sql}: $output") {
-          output.schema
-        }
-        assertResult(expected.output, s"Result did not match" +
-          s" for query #$i\n${expected.sql}") { output.output }
+      testCase match {
+        case _: AnalyzerTestCase =>
+          readGoldenFileAndCompareResults(testCase.resultFile, outputs, AnalyzerOutput)
+        case _ =>
+          readGoldenFileAndCompareResults(testCase.resultFile, outputs, ExecutionOutput)
       }
     }
   }
 
   protected lazy val listTestCases: Seq[TestCase] = {
     listFilesRecursively(new File(inputFilePath)).flatMap { file =>
       val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out"
+      val analyzerResultFile =
+        file.getAbsolutePath.replace(inputFilePath, analyzerGoldenFilePath) + ".out"
       val absPath = file.getAbsolutePath
       val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator)
+      val analyzerTestCaseName = s"${testCaseName}_analyzer_test"
 
+      val newTestCase =
       if (file.getAbsolutePath.startsWith(

Review Comment:
   ```suggestion
         val newTestCase = if (file.getAbsolutePath.startsWith(
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1476427922

   I'm AFK at this time. @gengliangwang can you help to revert it if @dtenedor can't fix the failure soon?


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139311712


##########
sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:
##########
@@ -228,6 +229,101 @@ abstract class QueryTest extends PlanTest {
     assert(query.queryExecution.executedPlan.missingInput.isEmpty,
       s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
   }
+
+  /** A single SQL query's output. */
+  protected trait QueryTestOutput {

Review Comment:
   Now all the changes in `QueryTest` is only used in `SQLQueryTestSuite`. Shall we move these changes to `SQLQueryTestSuite`?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1470824639

   @gengliangwang from past experience we will want to keep the query plans separate from the SQL results, otherwise the SQL results become hard to read. I will put the analyzer results in separate files.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139817260


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala:
##########
@@ -108,4 +126,23 @@ trait SQLQueryTestHelper {
         (emptySchema, Seq(e.getClass.getName, e.getMessage))
     }
   }
+
+  protected def splitWithSemicolon(seq: Seq[String]): Array[String] = {
+    seq.mkString("\n").split("(?<=[^\\\\]);")
+  }
+
+  protected def splitCommentsAndCodes(input: String): (Array[String], Array[String]) =
+    input.split("\n").partition { line =>
+      val newLine = line.trim
+      newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
+    }
+
+  /** Returns all the files (not directories) in a directory, recursively. */
+  protected def listFilesRecursively(path: File): Seq[File] = {

Review Comment:
   I assume these 3 method are just copied from the existing code



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1470823193

   @gengliangwang Sure, I was thinking about this too. We can reuse the same input SQL query files if we want, and just generate and test against different analyzer test output files. Let me update the PR to do that and I can ping the thread again.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139831906


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -266,22 +261,27 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       case _ =>
         // Create a test case to run this case.
         test(testCase.name) {
-          runTest(testCase)
+          runSqlTestCase(testCase, listTestCases, runQueries)
         }
     }
   }
 
-  /** Run a test case. */
-  protected def runTest(testCase: TestCase): Unit = {
-    def splitWithSemicolon(seq: Seq[String]) = {
-      seq.mkString("\n").split("(?<=[^\\\\]);")
-    }
-
-    def splitCommentsAndCodes(input: String) = input.split("\n").partition { line =>
-      val newLine = line.trim
-      newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
-    }
+  /** A test case. */
+  protected trait TestCase {
+    val name: String
+    val inputFile: String
+    val resultFile: String
+  }
 
+  /** Run a test case. */
+  protected def runSqlTestCase(
+      testCase: TestCase,
+      listTestCases: Seq[TestCase],
+      runQueries: (

Review Comment:
   why do we pass it as a parameter? We can directly invoke `runQueries` in `runSqlTestCase`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139833108


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -678,4 +663,99 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       super.afterAll()
     }
   }
+
+  /**
+   * Consumes contents from a single golden file and compares the expected results against the
+   * output of running a query.
+   */
+  def readGoldenFileAndCompareResults(
+    resultFile: String,

Review Comment:
   nit: 4 spaces indentation for parameters.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis
URL: https://github.com/apache/spark/pull/40449


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1476082725

   thanks, merging to master!


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1141100589


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala:
##########
@@ -108,4 +126,23 @@ trait SQLQueryTestHelper {
         (emptySchema, Seq(e.getClass.getName, e.getMessage))
     }
   }
+
+  protected def splitWithSemicolon(seq: Seq[String]): Array[String] = {
+    seq.mkString("\n").split("(?<=[^\\\\]);")
+  }
+
+  protected def splitCommentsAndCodes(input: String): (Array[String], Array[String]) =
+    input.split("\n").partition { line =>
+      val newLine = line.trim
+      newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
+    }
+
+  /** Returns all the files (not directories) in a directory, recursively. */
+  protected def listFilesRecursively(path: File): Seq[File] = {

Review Comment:
   Yes, I had moved these methods from their original place elsewhere in this file. I moved them back now to simplify the diff.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -338,6 +338,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       conf.trim -> value.substring(1).trim
     })
 
+    val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"

Review Comment:
   removed this.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -266,22 +261,27 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       case _ =>
         // Create a test case to run this case.
         test(testCase.name) {
-          runTest(testCase)
+          runSqlTestCase(testCase, listTestCases, runQueries)
         }
     }
   }
 
-  /** Run a test case. */
-  protected def runTest(testCase: TestCase): Unit = {
-    def splitWithSemicolon(seq: Seq[String]) = {
-      seq.mkString("\n").split("(?<=[^\\\\]);")
-    }
-
-    def splitCommentsAndCodes(input: String) = input.split("\n").partition { line =>
-      val newLine = line.trim
-      newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
-    }
+  /** A test case. */
+  protected trait TestCase {

Review Comment:
   Done.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -678,4 +663,99 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       super.afterAll()
     }
   }
+
+  /**
+   * Consumes contents from a single golden file and compares the expected results against the
+   * output of running a query.
+   */
+  def readGoldenFileAndCompareResults(
+    resultFile: String,
+    outputs: Seq[QueryTestOutput],
+    makeOutput: (String, Option[String], String) => QueryTestOutput): Unit = {
+    // Read back the golden file.
+    val expectedOutputs: Seq[QueryTestOutput] = {
+      val goldenOutput = fileToString(new File(resultFile))
+      val segments = goldenOutput.split("-- !query.*\n")
+
+      val numSegments = outputs.map(_.numSegments).sum + 1
+      assert(segments.size == numSegments,
+        s"Expected $numSegments blocks in result file but got " +
+          s"${segments.size}. Try regenerate the result files.")
+      var curSegment = 0
+      outputs.map { output =>
+        val result = if (output.numSegments == 3) {
+          makeOutput(
+            segments(curSegment + 1).trim, // SQL
+            Some(segments(curSegment + 2).trim), // Schema
+            segments(curSegment + 3).replaceAll("\\s+$", "")) // Output
+        } else {
+          makeOutput(
+            segments(curSegment + 1).trim, // SQL
+            None, // Schema
+            segments(curSegment + 2).replaceAll("\\s+$", "")) // Output
+        }
+        curSegment += output.numSegments
+        result
+      }
+    }
+
+    // Compare results.
+    assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") {
+      outputs.size
+    }
+
+    outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) =>
+      assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") {
+        output.sql
+      }
+      assertResult(expected.schema,
+        s"Schema did not match for query #$i\n${expected.sql}: $output") {
+        output.schema
+      }
+      assertResult(expected.output, s"Result did not match" +
+        s" for query #$i\n${expected.sql}") {
+        output.output
+      }
+    }
+  }
+
+  /** A single SQL query's output. */
+  trait QueryTestOutput {
+    def sql: String
+    def schema: Option[String]
+    def output: String
+    def numSegments: Int
+  }
+
+  /** A single SQL query's output. */

Review Comment:
   Done.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -678,4 +663,99 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       super.afterAll()
     }
   }
+
+  /**
+   * Consumes contents from a single golden file and compares the expected results against the
+   * output of running a query.
+   */
+  def readGoldenFileAndCompareResults(
+    resultFile: String,

Review Comment:
   Done.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -460,49 +473,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
     }
 
     withClue(clue) {
-      // Read back the golden file.
-      val expectedOutputs: Seq[QueryOutput] = {
-        val goldenOutput = fileToString(new File(testCase.resultFile))
-        val segments = goldenOutput.split("-- !query.*\n")
-
-        // each query has 3 segments, plus the header
-        assert(segments.size == outputs.size * 3 + 1,
-          s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " +
-            s"Try regenerate the result files.")
-        Seq.tabulate(outputs.size) { i =>
-          QueryOutput(
-            sql = segments(i * 3 + 1).trim,
-            schema = segments(i * 3 + 2).trim,
-            output = segments(i * 3 + 3).replaceAll("\\s+$", "")
-          )
-        }
-      }
-
-      // Compare results.
-      assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") {
-        outputs.size
-      }
-
-      outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) =>
-        assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") {
-          output.sql
-        }
-        assertResult(expected.schema,
-          s"Schema did not match for query #$i\n${expected.sql}: $output") {
-          output.schema
-        }
-        assertResult(expected.output, s"Result did not match" +
-          s" for query #$i\n${expected.sql}") { output.output }
+      testCase match {
+        case _: AnalyzerTestCase =>
+          readGoldenFileAndCompareResults(testCase.resultFile, outputs, AnalyzerOutput)
+        case _ =>
+          readGoldenFileAndCompareResults(testCase.resultFile, outputs, ExecutionOutput)
       }
     }
   }
 
   protected lazy val listTestCases: Seq[TestCase] = {
     listFilesRecursively(new File(inputFilePath)).flatMap { file =>
       val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out"
+      val analyzerResultFile =
+        file.getAbsolutePath.replace(inputFilePath, analyzerGoldenFilePath) + ".out"
       val absPath = file.getAbsolutePath
       val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator)
+      val analyzerTestCaseName = s"${testCaseName}_analyzer_test"
 
+      val newTestCase =
       if (file.getAbsolutePath.startsWith(

Review Comment:
   Done.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -266,22 +261,27 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       case _ =>
         // Create a test case to run this case.
         test(testCase.name) {
-          runTest(testCase)
+          runSqlTestCase(testCase, listTestCases, runQueries)
         }
     }
   }
 
-  /** Run a test case. */
-  protected def runTest(testCase: TestCase): Unit = {
-    def splitWithSemicolon(seq: Seq[String]) = {
-      seq.mkString("\n").split("(?<=[^\\\\]);")
-    }
-
-    def splitCommentsAndCodes(input: String) = input.split("\n").partition { line =>
-      val newLine = line.trim
-      newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
-    }
+  /** A test case. */
+  protected trait TestCase {
+    val name: String
+    val inputFile: String
+    val resultFile: String
+  }
 
+  /** Run a test case. */
+  protected def runSqlTestCase(
+      testCase: TestCase,
+      listTestCases: Seq[TestCase],
+      runQueries: (

Review Comment:
   Good point, I removed this and just call `runQueries` directly now.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1470712114

   Hi @gengliangwang this should be ready for a first look!


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139382943


##########
sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out:
##########


Review Comment:
   I expect there is `ansi/array.sql.out` as well



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139828611


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -266,22 +261,27 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       case _ =>
         // Create a test case to run this case.
         test(testCase.name) {
-          runTest(testCase)
+          runSqlTestCase(testCase, listTestCases, runQueries)
         }
     }
   }
 
-  /** Run a test case. */
-  protected def runTest(testCase: TestCase): Unit = {
-    def splitWithSemicolon(seq: Seq[String]) = {
-      seq.mkString("\n").split("(?<=[^\\\\]);")
-    }
-
-    def splitCommentsAndCodes(input: String) = input.split("\n").partition { line =>
-      val newLine = line.trim
-      newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
-    }
+  /** A test case. */
+  protected trait TestCase {

Review Comment:
   can we move it back to the original place? https://github.com/apache/spark/pull/40449/files#diff-212b35a074b1eeb518aff4c47130c7ab9cc23856ea6ea62f083af4452c637b9aL178



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1476470933

   Looks like some extra tests got added just as this was getting merged! Thanks @LuciferYang for this fix 👍


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1470971482

   @gengliangwang alright I made this change, please look again when you are ready.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139325632


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -205,6 +194,14 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
   protected case class RegularTestCase(
       name: String, inputFile: String, resultFile: String) extends TestCase
 
+  /** An ANSI-related test case. */
+  protected case class AnsiTestCase(
+      name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest
+
+  /** An analyzer test that shows the analyzed plan string as output. */
+  protected case class AnalyzerTestCase(

Review Comment:
   hmm, will we have analyzer-only test? Or you will remove this one when generating all the analyzer golden files?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139370077


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -205,6 +194,14 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
   protected case class RegularTestCase(
       name: String, inputFile: String, resultFile: String) extends TestCase
 
+  /** An ANSI-related test case. */
+  protected case class AnsiTestCase(
+      name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest
+
+  /** An analyzer test that shows the analyzed plan string as output. */
+  protected case class AnalyzerTestCase(

Review Comment:
   In this PR I made the test runner generate both a `RegularTestCase` and one of these new `AnalyzerTestCase` instances for each of the eligible input `.sql` files.



##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -525,23 +403,18 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
         AnsiTestCase(testCaseName, absPath, resultFile) :: Nil
       } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}timestampNTZ")) {
         TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil
+      } else if (analyzerTestCaseList.contains(file.getName.toLowerCase(Locale.ROOT))) {

Review Comment:
   Good idea, done!



##########
sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:
##########
@@ -228,6 +229,101 @@ abstract class QueryTest extends PlanTest {
     assert(query.queryExecution.executedPlan.missingInput.isEmpty,
       s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
   }
+
+  /** A single SQL query's output. */
+  protected trait QueryTestOutput {

Review Comment:
   Sure, sounds good! I moved it there.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139382539


##########
sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:
##########
@@ -228,6 +228,7 @@ abstract class QueryTest extends PlanTest {
     assert(query.queryExecution.executedPlan.missingInput.isEmpty,
       s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
   }
+

Review Comment:
   Let remove the unnecessary change



##########
sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:
##########
@@ -228,6 +228,7 @@ abstract class QueryTest extends PlanTest {
     assert(query.queryExecution.executedPlan.missingInput.isEmpty,
       s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}")
   }
+

Review Comment:
   Let's remove the unnecessary change



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139386978


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala:
##########
@@ -108,4 +128,132 @@ trait SQLQueryTestHelper {
         (emptySchema, Seq(e.getClass.getName, e.getMessage))
     }
   }
+
+  /** A test case. */
+  protected trait TestCase {
+    val name: String
+    val inputFile: String
+    val resultFile: String
+  }
+
+  /** Run a test case. */
+  protected def runTest(

Review Comment:
   Same here. Let's put the code used only in `SQLQueryTestSuite` back 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139831211


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -338,6 +338,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
       conf.trim -> value.substring(1).trim
     })
 
+    val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"

Review Comment:
   why do we redefine the variable here?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1476418435

   Seems GA break after this one:
   
   - https://github.com/apache/spark/actions/runs/4467843445/jobs/7847830877
   - https://github.com/apache/spark/actions/runs/4468075788/jobs/7848369978
   - https://github.com/apache/spark/actions/runs/4468868780/jobs/7850199229
   
   <img width="874" alt="image" src="https://user-images.githubusercontent.com/1475305/226384714-ee30f2c5-495c-4709-8a82-78361e1d35b3.png">
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1476437795

   Seems would be ok to regenerate the golden files
   
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on code in PR #40449:
URL: https://github.com/apache/spark/pull/40449#discussion_r1139327888


##########
sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala:
##########
@@ -525,23 +403,18 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
         AnsiTestCase(testCaseName, absPath, resultFile) :: Nil
       } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}timestampNTZ")) {
         TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil
+      } else if (analyzerTestCaseList.contains(file.getName.toLowerCase(Locale.ROOT))) {

Review Comment:
   I think analyzerTestCaseList should cover the `ansi`/`TimestampNTZ`/`postgreSQL` input queries as well



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on pull request #40449: [SPARK-42791][SQL] Create a new golden file test framework for analysis

Posted by "gengliangwang (via GitHub)" <gi...@apache.org>.
gengliangwang commented on PR #40449:
URL: https://github.com/apache/spark/pull/40449#issuecomment-1470835260

   > I will put the analyzer results in separate files.
   
   Sounds great! Thanks for the work!


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org