You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/09 07:17:22 UTC

[GitHub] [spark] turboFei opened a new pull request #29982: [SPARK-33110][SQL] Support parse the sql statements with c-style comments

turboFei opened a new pull request #29982:
URL: https://github.com/apache/spark/pull/29982


   ### What changes were proposed in this pull request?
   In this PR, we ignore the content in c-style comments while split the sql statements.
   
   ### Why are the changes needed?
   Spark-sql might split the statements inside c-style comments and it is not correct.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, it enable spark-sql support  parse the sql statements with c-style comments
   
   
   ### How was this patch tested?
   UT
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754335977


   **[Test build #133631 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133631/testReport)** for PR 29982 at commit [`f7f8030`](https://github.com/apache/spark/commit/f7f803034ae1ffa713b7015f214d63412e101389).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706554468






----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749961220


   > Could you add tests for the case? [#29982 (comment)](https://github.com/apache/spark/pull/29982#discussion_r509346762)
   
   Have added ut for it.
   Sorry for late reply, just com back from vacation.


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749978769


   > **[Test build #133268 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133268/testReport)** for PR 29982 at commit [`5a752d4`](https://github.com/apache/spark/commit/5a752d4a988a927323f70eab0b2d3e6b4720a3f5).
   > 
   > * This patch **fails to build**.
   > * This patch merges cleanly.
   > * This patch adds no public classes.
   
   
   Jenkins error:
   [error] java.io.IOException: No space left on device


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-710014843


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34498/
   Test FAILed.


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r509241110



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,14 +519,22 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore a semicolon inside a bracketed comment in spark-sql.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideSimpleComment = false
+    var bracketedCommentLevel = 0
     var escape = false
     var beginIndex = 0
+    var includingStatement = false
     val ret = new JArrayList[String]
 
+    def insideBracketedComment: Boolean = bracketedCommentLevel > 0

Review comment:
       Ah, ... I notice that handling bracketed comments is more complicated than I thought. For example, `SELECT /*+ COALESCE(3) */ * FROM t`. In this case, we need to pass the comment into the Spark parser.




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706496146


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34222/
   


----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551317016



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       Looks ok to me, but instead do we need to see what will happen if `;` is ignored wrongly while it's in the hint 




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706555650


   **[Test build #129628 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129628/testReport)** for PR 29982 at commit [`37c5fb6`](https://github.com/apache/spark/commit/37c5fb65de24f7e26c3d69d75a53ac2cdbf6d7c2).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706560944


   **[Test build #129628 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129628/testReport)** for PR 29982 at commit [`37c5fb6`](https://github.com/apache/spark/commit/37c5fb65de24f7e26c3d69d75a53ac2cdbf6d7c2).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753848107


   > Have you checked the comment: [#29982 (comment)](https://github.com/apache/spark/pull/29982#discussion_r551076053) ?
   
   Have addressed the comments.
   
   I use hint `MERGEJOIN` and only match the physical plan.
   So it is better to use `explain` instead of `explain extended`, but to prevent trigger one more jenkins job, I do not plan to remove the `extend`.
   
   


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754442509


   @turboFei Could you open a PR to fix it for branch-3.0?


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706563286






----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-748074780


   Could you add tests for the case? https://github.com/apache/spark/pull/29982#discussion_r509346762


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551093127



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,37 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val testHintTablePath = Utils.createTempDir()
+
+    runCliWithin(3.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2), (2, 3), (3, 4), (4, 5), (5, 6)" +
+        " as t1(key, val);"
+        -> "",
+      s"""CREATE TABLE testHint(key int, val int) USING hive
+        |LOCATION '${testHintTablePath.getAbsolutePath}';""".stripMargin
+        -> "",
+      "INSERT OVERWRITE TABLE testHint SELECT /*+ broadcast(t1) */ a.* from t1 a join t1 b" +
+        " on a.key=b.val;"
+        -> ""
+    )
+
+    val dataFiles = testHintTablePath.listFiles().filterNot{ file =>
+      file.getName.startsWith(".") || file.getName.startsWith("_")
+    }
+    assert(dataFiles.size == 1)
+
+    runCliWithin(1.minute)("DROP TABLE testHint;" -> "")

Review comment:
       sure, thanks




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753480199


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133593/
   


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r550201756



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       How about the case below? 
   ```
   select /*    + broadcast(t1) */ * from t1 join t2 on t1.id = t2.id;
            ^^^
            space, indent, ...
   ```




----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551318813



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       For example `EXPLAIN SELECT /* + MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;`, will the ';' be ignored?




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749971157


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133268/
   


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r552351281



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -573,8 +593,12 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
       } else if (line.charAt(index) == '\\') {
         escape = true
       }
+
+      includingStatement = statementBegin(index)

Review comment:
       Here is a bug.
   
   
   For the delimiter `;`, it would be treat as statementBegin




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551372031



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       sure, will add one




----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754320543


   **[Test build #133631 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133631/testReport)** for PR 29982 at commit [`f7f8030`](https://github.com/apache/spark/commit/f7f803034ae1ffa713b7015f214d63412e101389).


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753807873


   Have you checked the comment: https://github.com/apache/spark/pull/29982#discussion_r551076053 ?


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r509346762



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,14 +519,22 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore a semicolon inside a bracketed comment in spark-sql.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideSimpleComment = false
+    var bracketedCommentLevel = 0
     var escape = false
     var beginIndex = 0
+    var includingStatement = false
     val ret = new JArrayList[String]
 
+    def insideBracketedComment: Boolean = bracketedCommentLevel > 0

Review comment:
       yes, it would be passed to spark parser with this pr.
   
   Only comment without any statement content would be passed to spark parser.

##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,14 +519,22 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore a semicolon inside a bracketed comment in spark-sql.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideSimpleComment = false
+    var bracketedCommentLevel = 0
     var escape = false
     var beginIndex = 0
+    var includingStatement = false
     val ret = new JArrayList[String]
 
+    def insideBracketedComment: Boolean = bracketedCommentLevel > 0

Review comment:
       yes, it would be passed to spark parser with this pr.
   
   Only the comment without any statement content would be passed to spark parser.




----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503624175



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false

Review comment:
       `Dash` -> `Simple` https://github.com/apache/spark/blob/master/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4#L1816

##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without

Review comment:
       This statement looks ambiguous, so how about saying `Ignore a semicolon inside a bracketed comment in spark-sql`?

##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false

Review comment:
       Why do we need to track this state? `!insideComment` does not mean that a cursor exists in contents?

##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false

Review comment:
       nit: btw, `content` -> `statement`?

##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -550,21 +564,37 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
           // Sample query: select "quoted value --"
           //                                    ^^ avoids starting a comment if it's inside quotes.
         } else if (hasNext && line.charAt(index + 1) == '-') {
-          // ignore quotes and ;
-          insideComment = true
+          // ignore quotes and ; in dash-comment
+          insideDashComment = true
         }
       } else if (line.charAt(index) == ';') {
         if (insideSingleQuote || insideDoubleQuote || insideComment) {
           // do not split
         } else {
-          // split, do not include ; itself
-          ret.add(line.substring(beginIndex, index))
+          if (contentBegin) {
+            // split, do not include ; itself
+            ret.add(line.substring(beginIndex, index))
+          }
           beginIndex = index + 1
+          contentBegin = false
         }
       } else if (line.charAt(index) == '\n') {
-        // with a new line the inline comment should end.
+        // with a new line the inline dash-comment should end.
         if (!escape) {
-          insideComment = false
+          insideDashComment = false
+        }
+      } else if (line.charAt(index) == '/' && !insideComment) {
+        val hasNext = index + 1 < line.length
+        if (insideSingleQuote || insideDoubleQuote) {
+          // Ignores '/' in any case of quotes
+        } else if (hasNext && line.charAt(index + 1) == '*') {
+          // ignore quotes and ; in bracketed comment
+          insideBracketedComment = true
+        }
+      } else if (line.charAt(index) == '/' && insideBracketedComment) {
+        if (line.charAt(index - 1) == '*') {
+          // record the right bound of bracketed comment
+          bracketedCommentRightBound = index

Review comment:
       We need a variable `bracketedCommentRightBound` for tracking a bracketed comment?  I think we can do it without it.

##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -550,21 +564,37 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
           // Sample query: select "quoted value --"
           //                                    ^^ avoids starting a comment if it's inside quotes.
         } else if (hasNext && line.charAt(index + 1) == '-') {
-          // ignore quotes and ;
-          insideComment = true
+          // ignore quotes and ; in dash-comment

Review comment:
       nit: `// ignore quotes and semicolons in a simple comment`




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750023322


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37873/
   


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753869679


   seems no jenkins job triggered, I have passed the UT locally.
   gentle ping @maropu 


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502787723



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33110] Ignore the content inside C-style comment and ignore the comment without
+  // content
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideCStyleComment = false
+    var cStyleCommentRightBound = -1

Review comment:
       The name `cStyle` looks weird, so could you use `bracketedComment` instead?
   https://github.com/apache/spark/blob/master/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4#L1820-L1822




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706516444






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706563282


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34231/
   


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751624262


   test passed, gentle ping @maropu 


----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551364596



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       maybe we should add these to test?




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706636433


   **[Test build #129634 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129634/testReport)** for PR 29982 at commit [`9085cfd`](https://github.com/apache/spark/commit/9085cfd2a406e64c77c5e48506604259b85e8f23).


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551300441



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       nit format:
   ```
         "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);" -> "",
         "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);" -> "",
         "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;" -> "SortMergeJoin",
         "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
           -> "BroadcastHashJoin"
   ```




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706561007


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129628/
   Test FAILed.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751464568


   **[Test build #133407 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133407/testReport)** for PR 29982 at commit [`81d0215`](https://github.com/apache/spark/commit/81d021567808a5d950c1d8a8afce99cae33cf5f5).


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r550201387



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       Could you make  a new test unit for this test?




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750009205






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751424901


   **[Test build #133402 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133402/testReport)** for PR 29982 at commit [`39d38ca`](https://github.com/apache/spark/commit/39d38ca90ba71543f890a9b5edc8590e93d19f66).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706565650






----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750303470


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133308/
   


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503970282



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false

Review comment:
       There might no non-blank character outside comment.
   Like:
   ```
   -- comment
   \n
   \n
   \n
   ```




----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755071408


   create https://github.com/apache/spark/pull/31054 to fix this issue


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709999394






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708939495


   **[Test build #129811 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129811/testReport)** for PR 29982 at commit [`90fb234`](https://github.com/apache/spark/commit/90fb2348f0afa0f3a56a0cf19b00c78b0cabd9ec).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751491341


   **[Test build #133411 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133411/testReport)** for PR 29982 at commit [`55aa0c3`](https://github.com/apache/spark/commit/55aa0c3beb989933437ab81b5e10e9564f8046d2).


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503936995



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without

Review comment:
       good idea




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r552351281



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -573,8 +593,12 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
       } else if (line.charAt(index) == '\\') {
         escape = true
       }
+
+      includingStatement = statementBegin(index)

Review comment:
       Here is a bug.
   
   
   For the delimiter `;`, it would be treat as beginning of statement




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751603764


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133427/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708946583


   **[Test build #129811 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129811/testReport)** for PR 29982 at commit [`90fb234`](https://github.com/apache/spark/commit/90fb2348f0afa0f3a56a0cf19b00c78b0cabd9ec).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r509346762



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,14 +519,22 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore a semicolon inside a bracketed comment in spark-sql.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideSimpleComment = false
+    var bracketedCommentLevel = 0
     var escape = false
     var beginIndex = 0
+    var includingStatement = false
     val ret = new JArrayList[String]
 
+    def insideBracketedComment: Boolean = bracketedCommentLevel > 0

Review comment:
       yes, it would be passed to spark parser with this pr.
   
   Only the comment without any statement content would not be passed to spark parser.




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754336507


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133631/
   


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753959684


   Looks fine cc: @HyukjinKwon @yaooqinn @wangyum


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503949065



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -550,21 +564,37 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
           // Sample query: select "quoted value --"
           //                                    ^^ avoids starting a comment if it's inside quotes.
         } else if (hasNext && line.charAt(index + 1) == '-') {
-          // ignore quotes and ;
-          insideComment = true
+          // ignore quotes and ; in dash-comment

Review comment:
       thanks




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754418184


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38220/
   


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751498352


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38002/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706222990


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34190/
   


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503948938



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false

Review comment:
       It is a little difficult to recognize whether the statement begin.
   For example:
   ```
   /*  comment */
   \n
   ```
   The "\n" is outside comment, but there is no statement.




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706497848






----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708702280


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706495632


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129616/
   Test FAILed.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753480199


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133593/
   


----------------------------------------------------------------
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.

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] gatorsmile commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755063398


   CC @bogdanghit 


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706497848






----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551376467



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }

Review comment:
       done




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751422751


   **[Test build #133402 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133402/testReport)** for PR 29982 at commit [`39d38ca`](https://github.com/apache/spark/commit/39d38ca90ba71543f890a9b5edc8590e93d19f66).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708601869


   **[Test build #129759 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129759/testReport)** for PR 29982 at commit [`2f48be5`](https://github.com/apache/spark/commit/2f48be57393135fbb91712cc8f6ba3309145d3c7).


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503628406



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false
     val ret = new JArrayList[String]
 
+    def insideComment: Boolean = insideDashComment || insideBracketedComment
+    def isContent(char: Char): Boolean = !insideComment && !s"$char".trim.isEmpty

Review comment:
       `!s"$char".trim.isEmpty` -> `char != ' '`?




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502792787



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33110] Ignore the content inside C-style comment and ignore the comment without
+  // content

Review comment:
       thanks




----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706182552






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709343976


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34443/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709925226


   **[Test build #129888 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129888/testReport)** for PR 29982 at commit [`085d4b9`](https://github.com/apache/spark/commit/085d4b97e40dda6a9e9b6afff6292060b90620a4).


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754444555


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38220/
   


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754472646


   > @turboFei Could you open a PR to fix it for branch-3.0/2.4?
   
   sure


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551076912



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,37 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val testHintTablePath = Utils.createTempDir()
+
+    runCliWithin(3.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2), (2, 3), (3, 4), (4, 5), (5, 6)" +
+        " as t1(key, val);"
+        -> "",
+      s"""CREATE TABLE testHint(key int, val int) USING hive

Review comment:
       I meant, "could you avoid creating a catalog table for this test?" https://github.com/apache/spark/pull/29982#discussion_r550202290
   We cannot add a test like this?
   ```
   EXPLAIN EXTENDED SELECT /*+ broadcast(t1) */ * from t1
   // Then, check if an explain result has `ResolvedHint (strategy=broadcast)`
   == Analyzed Logical Plan ==
   k: int, v: int
   Project [k#0, v#1]
   +- ResolvedHint (strategy=broadcast)
      ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
      +- SubqueryAlias t1
         +- Project [k#0, v#1]
            +- SubqueryAlias t1
               +- LocalRelation [k#0, v#1]
   ```




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706560972


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34231/
   


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502791185



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,10 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33110: Support parse the sql statements with c-style comments") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test'" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")

Review comment:
       Could you add more test cases? e.g., multiple `;` and comment before/after statement `/* XXX; */ SELECT 1 /* YYY; */`




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751469051


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37998/
   


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706491725






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706565645


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34232/
   


----------------------------------------------------------------
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.

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] maropu edited a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706550342


   cc: @wangyum @yaooqinn 


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709942664


   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129888/
   Test PASSed.


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708637490






----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709335995






----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751464568


   **[Test build #133407 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133407/testReport)** for PR 29982 at commit [`81d0215`](https://github.com/apache/spark/commit/81d021567808a5d950c1d8a8afce99cae33cf5f5).


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753486813


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38182/
   


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708939495


   **[Test build #129811 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129811/testReport)** for PR 29982 at commit [`90fb234`](https://github.com/apache/spark/commit/90fb2348f0afa0f3a56a0cf19b00c78b0cabd9ec).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751469487


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37998/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708636899


   **[Test build #129759 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129759/testReport)** for PR 29982 at commit [`2f48be5`](https://github.com/apache/spark/commit/2f48be57393135fbb91712cc8f6ba3309145d3c7).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551173235



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       plan for  `select /*    + broadcast(t) */ * from t`
   
   ```
   == Parsed Logical Plan ==
   'Project [*]
   +- 'UnresolvedRelation [t], [], false
   
   == Analyzed Logical Plan ==
   k: int, v: int
   Project [k#218, v#219]
   +- SubqueryAlias t
      +- Project [k#218, v#219]
         +- SubqueryAlias t
            +- LocalRelation [k#218, v#219]
   
   == Optimized Logical Plan ==
   LocalRelation [k#218, v#219]
   
   == Physical Plan ==
   LocalTableScan [k#218, v#219]
   ```
   
   
   I do not find the way to match result for this case.
   Because the `captureOutput` method capture output from source `output` one line by one line.
   
   It can not match two  continue lines.




----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551304345



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"

Review comment:
       NOTE: This test can pass even in the current master, but adding this test looks fine since `CliSuite` does not have tests now for bracketed comments with hints, I think.




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706017122


   Can one of the admins verify this patch?


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749988953


   **[Test build #133275 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133275/testReport)** for PR 29982 at commit [`5cbd308`](https://github.com/apache/spark/commit/5cbd30850b1403e34e489ba1adbbafc8cf82894a).


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551679300



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"

Review comment:
       got it, thanks




----------------------------------------------------------------
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.

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] maropu edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754442509


   @turboFei Could you open a PR to fix it for branch-3.0/2.4?


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551341615



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       sure, the `;` ` /* + MERGEJOIN(t1); */` in  will be ignored

##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       sure, the `;` in ` /* + MERGEJOIN(t1); */` in  will be ignored




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751621035


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38018/
   


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751496932






----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709952617






----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709925226


   **[Test build #129888 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129888/testReport)** for PR 29982 at commit [`085d4b9`](https://github.com/apache/spark/commit/085d4b97e40dda6a9e9b6afff6292060b90620a4).


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551109247



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,37 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val testHintTablePath = Utils.createTempDir()
+
+    runCliWithin(3.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2), (2, 3), (3, 4), (4, 5), (5, 6)" +
+        " as t1(key, val);"
+        -> "",
+      s"""CREATE TABLE testHint(key int, val int) USING hive

Review comment:
       thanks a lot, I thought the answer match is absolute total match, so I did not use the logical plan match.
   
   Thanks a lot, have addressed it.




----------------------------------------------------------------
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.

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] maropu removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706550342


   cc: @wangyum @yaooqinn 


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706491002


   **[Test build #129616 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129616/testReport)** for PR 29982 at commit [`6b225ec`](https://github.com/apache/spark/commit/6b225ec9bacb34e6f911f4903050648b58211b5b).


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708379192


   > So, I recommend to save this variable.
   
   For example, I thought it like this (this approach also can pass the added test in this PR): https://github.com/maropu/spark/commit/d9c256ad32b5ad5bce419d1eed0e42ff764c3b4c
   Probably, we need more tests for better branch coverage, e.g., the case you described in https://github.com/apache/spark/pull/29982#discussion_r503948938, bracketed comments having meta chars, multi-line bracketed comments, brabrabra....


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709984531


   **[Test build #129893 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129893/testReport)** for PR 29982 at commit [`d24a00d`](https://github.com/apache/spark/commit/d24a00de7ee5352768197568fc8bcc5ff48f2194).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751589154


   **[Test build #133427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133427/testReport)** for PR 29982 at commit [`438c1cb`](https://github.com/apache/spark/commit/438c1cb2d559aa8e31f2284dc2c79471f0c18044).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706520449


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34225/
   


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751426097


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133402/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706641279


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34238/
   


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706204268






----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-744200742


   Sorry, before I reinstalled my mac os and missed this pr.
   
   
   gentle ping @maropu 


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753474740


   **[Test build #133593 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133593/testReport)** for PR 29982 at commit [`e61ad4d`](https://github.com/apache/spark/commit/e61ad4d7c0037f3a724e81af5697f8c00b2caa15).


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755054849


   The rc should be that:
   
   These two statements only return one result.
   
   Might the first statement contains an invalid statement `/* SELECT 'test';*/` and does not return result, or these two statements execute concurrently and conflict each other.
   
   ![image](https://user-images.githubusercontent.com/6757692/103725777-46a84b80-5012-11eb-9df9-dec105215259.png)
   
   
   I think we can ignore ignore the comments during two `;`.


----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551343441



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       So `/* + MERGEJOIN(t1); */ ` === `/* + MERGEJOIN(t1) */`? I am not very familiar with hints




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706520455






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706516368


   **[Test build #129621 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129621/testReport)** for PR 29982 at commit [`bc1bb37`](https://github.com/apache/spark/commit/bc1bb37cd33a16ca39d31a90e2b50815cf75b094).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750039768


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37873/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749971136


   **[Test build #133268 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133268/testReport)** for PR 29982 at commit [`5a752d4`](https://github.com/apache/spark/commit/5a752d4a988a927323f70eab0b2d3e6b4720a3f5).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709344012






----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709336010


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129838/
   Test FAILed.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706495625






----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551343441



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       So `/*+ MERGEJOIN(t1); */ ` === `/*+ MERGEJOIN(t1) */`? I am not very familiar with hints




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551158253



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       sorry, I will add a test for this case.




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708702287


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34365/
   Test FAILed.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750039768


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37873/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751601255


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38018/
   


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709984531


   **[Test build #129893 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129893/testReport)** for PR 29982 at commit [`d24a00d`](https://github.com/apache/spark/commit/d24a00de7ee5352768197568fc8bcc5ff48f2194).


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551173235



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       plan for  `select /*    + broadcast(t) */ * from t`
   
   ```
   == Parsed Logical Plan ==
   'Project [*]
   +- 'UnresolvedRelation [t], [], false
   
   == Analyzed Logical Plan ==
   k: int, v: int
   Project [k#218, v#219]
   +- SubqueryAlias t
      +- Project [k#218, v#219]
         +- SubqueryAlias t
            +- LocalRelation [k#218, v#219]
   
   == Optimized Logical Plan ==
   LocalRelation [k#218, v#219]
   
   == Physical Plan ==
   LocalTableScan [k#218, v#219]
   ```
   
   
   I can not find the way to match result for this case.
   Because the `captureOutput` method capture output from source `output` one line by one line.
   
   It can not match two  continue lines.




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751603764


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133427/
   


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753474670


   have refactored ut


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751468467


   **[Test build #133407 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133407/testReport)** for PR 29982 at commit [`81d0215`](https://github.com/apache/spark/commit/81d021567808a5d950c1d8a8afce99cae33cf5f5).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706562373


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34232/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753479256


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38182/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754431986


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38220/
   


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755054849


   These two statements only return one result.
   
   Might the first statement contains an invalid statement `/* SELECT 'test';*/` and does not return result.
   
   ![image](https://user-images.githubusercontent.com/6757692/103725777-46a84b80-5012-11eb-9df9-dec105215259.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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751493023


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38001/
   


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709978338


   Seems the bracketedCommentLevel could not handle this case:
   ```
   /* /* */ */
   ```
   
   ![image](https://user-images.githubusercontent.com/6757692/96250933-3b7ceb00-0fe2-11eb-9a47-ef7acb4ad311.png)
   
   WIP


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709328721


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34443/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706495404


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34219/
   


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751470398






----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551076053



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       Have you added tests for the case above?




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751621035


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38018/
   


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-744200742


   Sorry, before I reinstall my mac os and miss this pr.
   
   
   gentle ping @maropu 


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-707801924


   thanks for your comments @maropu @yaooqinn 
   I tried to thought how to remove the variable `bracketedCommentRightBound` but could not find a better way to recognize bracketed comment.
   
   Without this variable:
   1. I might need leave the bracketed comment when current char equals '/' and last char equals '*', but it is difficult to judge whether the statement begin.
   2. If I do not leave the bracketed comment when current char equals '/' and last char equals '*', it is difficult to leave the comment later.
   For example:
   the line likes: select */*comment\*/ from ta;
   
   So, I recommend to save this variable.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749968433


   **[Test build #133268 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133268/testReport)** for PR 29982 at commit [`5a752d4`](https://github.com/apache/spark/commit/5a752d4a988a927323f70eab0b2d3e6b4720a3f5).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709942293


   **[Test build #129888 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129888/testReport)** for PR 29982 at commit [`085d4b9`](https://github.com/apache/spark/commit/085d4b97e40dda6a9e9b6afff6292060b90620a4).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706559990






----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551375840



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       It seems that ` /*+ MERGEJOIN(t1); */` does not conform the syntax.
   
   ```
   [info]   2021-01-04 07:09:03.961 - stdout> spark-sql> EXPLAIN SELECT /*+ MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;
   [info]   2021-01-04 07:09:03.967 - stderr> Error in query: 
   [info]   2021-01-04 07:09:03.967 - stderr> extraneous input ';' expecting {',', '*/', 'ADD', 'AFTER', 'ALL', 'ALTER', 'ANALYZE', 'AND', 'ANTI', 'ANY', 'ARCHIVE', 'ARRAY', 'AS', 'ASC', 'AT', 'AUTHORIZATION', 'BETWEEN', 'BOTH', 'BUCKET', 'BUCKETS', 'BY', 'CACHE', 'CASCADE', 'CASE', 'CAST', 'CHANGE', 'CHECK', 'CLEAR', 'CLUSTER', 'CLUSTERED', 'CODEGEN', 'COLLATE', 'COLLECTION', 'COLUMN', 'COLUMNS', 'COMMENT', 'COMMIT', 'COMPACT', 'COMPACTIONS', 'COMPUTE', 'CONCATENATE', 'CONSTRAINT', 'COST', 'CREATE', 'CROSS', 'CUBE', 'CURRENT', 'CURRENT_DATE', 'CURRENT_TIME', 'CURRENT_TIMESTAMP', 'CURRENT_USER', 'DATA', 'DATABASE', DATABASES, 'DBPROPERTIES', 'DEFINED', 'DELETE', 'DELIMITED', 'DESC', 'DESCRIBE', 'DFS', 'DIRECTORIES', 'DIRECTORY', 'DISTINCT', 'DISTRIBUTE', 'DIV', 'DROP', 'ELSE', 'END', 'ESCAPE', 'ESCAPED', 'EXCEPT', 'EXCHANGE', 'EXISTS', 'EXPLAIN', 'EXPORT', 'EXTENDED', 'EXTERNAL', 'EXTRACT', 'FALSE', 'FETCH', 'FIELDS', 'FILTER', 'FILEFORMAT', 'FIRST', 'FOLLOWING', 'FOR', 'FOREIGN', 'F
 ORMAT', 'FORMATTED', 'FROM', 'FULL', 'FUNCTION', 'FUNCTIONS', 'GLOBAL', 'GRANT', 'GROUP', 'GROUPING', 'HAVING', 'IF', 'IGNORE', 'IMPORT', 'IN', 'INDEX', 'INDEXES', 'INNER', 'INPATH', 'INPUTFORMAT', 'INSERT', 'INTERSECT', 'INTERVAL', 'INTO', 'IS', 'ITEMS', 'JOIN', 'KEYS', 'LAST', 'LATERAL', 'LAZY', 'LEADING', 'LEFT', 'LIKE', 'LIMIT', 'LINES', 'LIST', 'LOAD', 'LOCAL', 'LOCATION', 'LOCK', 'LOCKS', 'LOGICAL', 'MACRO', 'MAP', 'MATCHED', 'MERGE', 'MSCK', 'NAMESPACE', 'NAMESPACES', 'NATURAL', 'NO', NOT, 'NULL', 'NULLS', 'OF', 'ON', 'ONLY', 'OPTION', 'OPTIONS', 'OR', 'ORDER', 'OUT', 'OUTER', 'OUTPUTFORMAT', 'OVER', 'OVERLAPS', 'OVERLAY', 'OVERWRITE', 'PARTITION', 'PARTITIONED', 'PARTITIONS', 'PERCENT', 'PIVOT', 'PLACING', 'POSITION', 'PRECEDING', 'PRIMARY', 'PRINCIPALS', 'PROPERTIES', 'PURGE', 'QUERY', 'RANGE', 'RECORDREADER', 'RECORDWRITER', 'RECOVER', 'REDUCE', 'REFERENCES', 'REFRESH', 'RENAME', 'REPAIR', 'REPLACE', 'RESET', 'RESTRICT', 'REVOKE', 'RIGHT', RLIKE, 'ROLE', 'ROLES', 'ROLLBACK
 ', 'ROLLUP', 'ROW', 'ROWS', 'SCHEMA', 'SELECT', 'SEMI', 'SEPARATED', 'SERDE', 'SERDEPROPERTIES', 'SESSION_USER', 'SET', 'MINUS', 'SETS', 'SHOW', 'SKEWED', 'SOME', 'SORT', 'SORTED', 'START', 'STATISTICS', 'STORED', 'STRATIFY', 'STRUCT', 'SUBSTR', 'SUBSTRING', 'TABLE', 'TABLES', 'TABLESAMPLE', 'TBLPROPERTIES', TEMPORARY, 'TERMINATED', 'THEN', 'TIME', 'TO', 'TOUCH', 'TRAILING', 'TRANSACTION', 'TRANSACTIONS', 'TRANSFORM', 'TRIM', 'TRUE', 'TRUNCATE', 'TYPE', 'UNARCHIVE', 'UNBOUNDED', 'UNCACHE', 'UNION', 'UNIQUE', 'UNKNOWN', 'UNLOCK', 'UNSET', 'UPDATE', 'USE', 'USER', 'USING', 'VALUES', 'VIEW', 'VIEWS', 'WHEN', 'WHERE', 'WINDOW', 'WITH', 'ZONE', IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 32)
   [info]   2021-01-04 07:09:03.967 - stderr> 
   [info]   2021-01-04 07:09:03.967 - stderr> == SQL ==
   [info]   2021-01-04 07:09:03.967 - stderr> EXPLAIN SELECT /*+ MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v
   [info]   2021-01-04 07:09:03.967 - stderr> --------------------------------^^^
   [info]   2021-01-04 07:09:03.968 - stderr> 
   [info]   2021-01-04 07:09:04.102 - stdout> spark-sql> 
   
   ```




----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551317842



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }

Review comment:
       How about grouping these queries and expected answers in a single `runCliWithin` to create 8 sub procs?




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754336507


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133631/
   


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706223010






----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503970282



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false

Review comment:
       There might no non-blank character outside comment.  
   Like:
   ```
   -- comment
   \n
   \n
   \n
   ```




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502792840



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,10 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33110: Support parse the sql statements with c-style comments") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test'" -> "")

Review comment:
       this test is used for ignoring the comment without content




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708973128


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34416/
   


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706512166


   **[Test build #129621 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129621/testReport)** for PR 29982 at commit [`bc1bb37`](https://github.com/apache/spark/commit/bc1bb37cd33a16ca39d31a90e2b50815cf75b094).


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r550877002



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")
+    val testHintTablePath = Utils.createTempDir()
+
+    runCliWithin(3.minute)(
+      "CREATE TABLE test(key INT, val STRING) USING hive;"

Review comment:
       temporary table is not supported now, I will try to use temporary view




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706204268






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706559941


   **[Test build #129627 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129627/testReport)** for PR 29982 at commit [`173f426`](https://github.com/apache/spark/commit/173f4260327412d073fa995a4a8409d048449f71).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-710014809


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34498/
   


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706550342


   cc: @wangyum


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r552351281



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -573,8 +593,12 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
       } else if (line.charAt(index) == '\\') {
         escape = true
       }
+
+      includingStatement = statementBegin(index)

Review comment:
       Here is a bug.
   
   
   For the delimiter `;`, it would be treat as beginning of statement




----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503630325



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,14 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore the content inside bracketed comment and ignore the comment without" +
+    " content.") {
+    runCliWithin(1.minute)("/* SELECT 1;*/ SELECT 1;" -> "1" )
+    runCliWithin(1.minute)(";;/* SELECT 1;*/ SELECT 1;" -> "1" )
+    runCliWithin(1.minute)("/* SELECT 1;*/;; SELECT 1;" -> "1" )

Review comment:
       These cases are not good, `1` is not distinguishable as an expected answer. 




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708973146






----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503951186



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideBracketedComment = false
+    var bracketedCommentRightBound = -1
     var escape = false
     var beginIndex = 0
+    var contentBegin = false
     val ret = new JArrayList[String]
 
+    def insideComment: Boolean = insideDashComment || insideBracketedComment
+    def isContent(char: Char): Boolean = !insideComment && !s"$char".trim.isEmpty

Review comment:
       this character might be a "\t" or "\n".




----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755064419


   there is a bug for statementBegin method.
   For `/* SELECT 'test';*/`,  the last character `/` would be treated as beginning of statements


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750305981


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37902/
   


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755064419


   there is a bug for statementBegin method.
   For `/* SELECT 'test';*/`, for the last character `/`, it would be treated as beginning of statements


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-749968433


   **[Test build #133268 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133268/testReport)** for PR 29982 at commit [`5a752d4`](https://github.com/apache/spark/commit/5a752d4a988a927323f70eab0b2d3e6b4720a3f5).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750001732


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37867/
   


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r550202290



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")
+    val testHintTablePath = Utils.createTempDir()
+
+    runCliWithin(3.minute)(
+      "CREATE TABLE test(key INT, val STRING) USING hive;"

Review comment:
       nit: Could you use a temporary table + a broadcast hint instead for better test performance?




----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753853447


   seems jenkins job not trigged, make a new commit to remove `extended`


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755064419


   there should be a bug for `statementBegin`


----------------------------------------------------------------
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.

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] maropu commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502788007



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,10 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33110: Support parse the sql statements with c-style comments") {

Review comment:
       Could you update the test title, too?




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503936583



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33100] Ignore the content inside bracketed comment and ignore the comment without
+  // content.
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false

Review comment:
       thanks




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751470399






----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706017661


   Can one of the admins verify this patch?


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751498347


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38002/
   


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706561003






----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706639219






----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706636433


   **[Test build #129634 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129634/testReport)** for PR 29982 at commit [`9085cfd`](https://github.com/apache/spark/commit/9085cfd2a406e64c77c5e48506604259b85e8f23).


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706641284






----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708637490






----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502792899



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,10 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33110: Support parse the sql statements with c-style comments") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test'" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")

Review comment:
       sure




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706017122






----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708946650






----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753491381


   gentle ping @maropu 


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753486813


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38182/
   


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709999394






----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709304676


   **[Test build #129838 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129838/testReport)** for PR 29982 at commit [`7aee369`](https://github.com/apache/spark/commit/7aee3694ec1b1ccb619164714da37b32237a8214).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with bracketed comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706554468


   **[Test build #129627 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129627/testReport)** for PR 29982 at commit [`173f426`](https://github.com/apache/spark/commit/173f4260327412d073fa995a4a8409d048449f71).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709304676


   **[Test build #129838 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129838/testReport)** for PR 29982 at commit [`7aee369`](https://github.com/apache/spark/commit/7aee3694ec1b1ccb619164714da37b32237a8214).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-710005412


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34498/
   


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551178126



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       will try to use `MERGEJOIN(t)` to verify broadcastjoin or mergejoin




----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750009204






----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503950372



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,14 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore the content inside bracketed comment and ignore the comment without" +
+    " content.") {
+    runCliWithin(1.minute)("/* SELECT 1;*/ SELECT 1;" -> "1" )
+    runCliWithin(1.minute)(";;/* SELECT 1;*/ SELECT 1;" -> "1" )
+    runCliWithin(1.minute)("/* SELECT 1;*/;; SELECT 1;" -> "1" )
+    runCliWithin(1.minute)("SELECT '1'; -- SELECT '1';" -> "")
+    runCliWithin(1.minute)("SELECT '1'; /* SELECT '1';*/" -> "")
+    runCliWithin(1.minute)("/* SELECT 1;*/;; SELECT 1; /* SELECT '1';*/" -> "" )

Review comment:
       It seems this case is redundant, will remove it.




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709942655






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706184098






----------------------------------------------------------------
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.

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] maropu edited a comment on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706182552


   If a comment `/* ... */` includes `;`. it will throw an exception? The comment style is already supported, so the title and the description look confusing. I think this is just a bug.


----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709978338


   Seems the bracketedCommentLevel could handle this case:
   ```
   /* /* */ */
   ```
   
   ![image](https://user-images.githubusercontent.com/6757692/96250933-3b7ceb00-0fe2-11eb-9a47-ef7acb4ad311.png)
   
   
   WIP


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706265133


   @maropu  Thanks for your comments, I have modified the title.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754444555


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38220/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754320543


   **[Test build #133631 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133631/testReport)** for PR 29982 at commit [`f7f8030`](https://github.com/apache/spark/commit/f7f803034ae1ffa713b7015f214d63412e101389).


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706223010






----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551318813



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       For example `EXPLAIN SELECT /*+ MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;`, will the ';' be ignored?




----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751484066


   **[Test build #133410 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133410/testReport)** for PR 29982 at commit [`f1befd3`](https://github.com/apache/spark/commit/f1befd3ab5cfcbfab1cbcd07b09396e4ecae5212).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750320217


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37902/
   


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-707801924


   thanks for your comments @maropu @yaooqinn 
   I tried to thought how to remove the variable `bracketedCommentRightBound` but could not find a better way to recognize bracketed comment.
   
   Without this variable:
   1. I might need leave the bracketed comment '/* */' when the char equals '/' and last char equals '*', but it is difficult to judge whether the statement begin.
   2. If I do not leave the bracketed comment when the char equals '/' and last char equals '*', it is difficult to leave the comment later.
   Such as the line likes: select */*comment*/ from ta;
   
   So, I recommend to save this variable.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706495569






----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706184098


   **[Test build #129585 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129585/testReport)** for PR 29982 at commit [`084fc37`](https://github.com/apache/spark/commit/084fc3757fdd3648be1ead88177124d7b9df6941).


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750303470


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133308/
   


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751589154


   **[Test build #133427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133427/testReport)** for PR 29982 at commit [`438c1cb`](https://github.com/apache/spark/commit/438c1cb2d559aa8e31f2284dc2c79471f0c18044).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751495034


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38002/
   


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709999006


   **[Test build #129893 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129893/testReport)** for PR 29982 at commit [`d24a00d`](https://github.com/apache/spark/commit/d24a00de7ee5352768197568fc8bcc5ff48f2194).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708924508


   >  For example, I thought it like this (this approach also can pass the added test in this PR): maropu@d9c256a
   
   Good idea. Thanks a lot.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708702264


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34365/
   


----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r503630451



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,14 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore the content inside bracketed comment and ignore the comment without" +
+    " content.") {
+    runCliWithin(1.minute)("/* SELECT 1;*/ SELECT 1;" -> "1" )
+    runCliWithin(1.minute)(";;/* SELECT 1;*/ SELECT 1;" -> "1" )
+    runCliWithin(1.minute)("/* SELECT 1;*/;; SELECT 1;" -> "1" )
+    runCliWithin(1.minute)("SELECT '1'; -- SELECT '1';" -> "")
+    runCliWithin(1.minute)("SELECT '1'; /* SELECT '1';*/" -> "")
+    runCliWithin(1.minute)("/* SELECT 1;*/;; SELECT 1; /* SELECT '1';*/" -> "" )

Review comment:
       What do we try to capture 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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709942655


   Merged build finished. Test PASSed.


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r502792807



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -519,15 +519,29 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
   // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted
   // string, the origin implementation from Hive will not drop the trailing semicolon as expected,
   // hence we refined this function a little bit.
+  // Note: [SPARK-33110] Ignore the content inside C-style comment and ignore the comment without
+  // content
   private def splitSemiColon(line: String): JList[String] = {
     var insideSingleQuote = false
     var insideDoubleQuote = false
-    var insideComment = false
+    var insideDashComment = false
+    var insideCStyleComment = false
+    var cStyleCommentRightBound = -1

Review comment:
       thanks 




----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753869679


   seems no jenkins job triggered, I have passed the UI locally.
   gentle ping @maropu 


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754954284


   @turboFei I found some GA flakiness caused by this commit, e.g., 
   https://github.com/apache/spark/pull/31045/checks?check_run_id=1652972350
   https://github.com/apache/spark/runs/1652975825?check_suite_focus=true
   
   Could you check/fix it? FYI: @dongjoon-hyun @HyukjinKwon 


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706498098


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34222/
   


----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551380618



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       That's expected behavior and not be affected by this 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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751426845


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37993/
   


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706550390


   cc: @wangyum @yaooqinn


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706639140


   **[Test build #129634 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129634/testReport)** for PR 29982 at commit [`9085cfd`](https://github.com/apache/spark/commit/9085cfd2a406e64c77c5e48506604259b85e8f23).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706498104






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Fix the issue when parsing sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706512166


   **[Test build #129621 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129621/testReport)** for PR 29982 at commit [`bc1bb37`](https://github.com/apache/spark/commit/bc1bb37cd33a16ca39d31a90e2b50815cf75b094).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751426840


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37993/
   


----------------------------------------------------------------
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.

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] maropu edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753959684


   Thanks for fixing this, @turboFei. Looks fine cc: @HyukjinKwon @yaooqinn @wangyum


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-708694182


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34365/
   


----------------------------------------------------------------
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.

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] maropu commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-754983375


   What's a root cause of the flakiness? It depends on the cause, I think.


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709335446


   **[Test build #129838 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129838/testReport)** for PR 29982 at commit [`7aee369`](https://github.com/apache/spark/commit/7aee3694ec1b1ccb619164714da37b32237a8214).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-710014833


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751490521


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133410/
   


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551173235



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       plan for  `select /*    + broadcast(t) */ * from t`
   
   ```
   == Parsed Logical Plan ==
   'Project [*]
   +- 'UnresolvedRelation [t], [], false
   
   == Analyzed Logical Plan ==
   k: int, v: int
   Project [k#218, v#219]
   +- SubqueryAlias t
      +- Project [k#218, v#219]
         +- SubqueryAlias t
            +- LocalRelation [k#218, v#219]
   
   == Optimized Logical Plan ==
   LocalRelation [k#218, v#219]
   
   == Physical Plan ==
   LocalTableScan [k#218, v#219]
   ```
   
   
   I can not find the way to match result for this case.
   Because the `captureOutput` method captures output from source `output` one line by one line.
   
   It can not match two  continue lines.




----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r550876621



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,53 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test" )
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+
+    val dataFilePath =
+      Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")

Review comment:
       sure, will try it




----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore the content inside bracketed comment and ignore the comment without content

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706559990






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751489171


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38001/
   


----------------------------------------------------------------
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.

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] turboFei commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753848107


   > Have you checked the comment: [#29982 (comment)](https://github.com/apache/spark/pull/29982#discussion_r551076053) ?
   
   Have addressed the comments.
   
   I use hint `MERGEJOIN` and only match the physical plan.
   So it is better to use `explain query` instead of `explain extended`, but to prevent trigger one more jenkins job, I do not plan to remove the `extend`.
   
   


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551362824



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       no,  the ; in /* + MERGEJOIN(t1); */ in will be ignored means that the `;` would be treated as a delimiter, which used to split sql statements.
   
   
   `EXPLAIN SELECT /*+ MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v` would be transfered to SparkSQLCliDriver totally.




----------------------------------------------------------------
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.

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] yaooqinn commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551318813



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       For example EXPLAIN SELECT /* + MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v, will the ';' be ignored?




----------------------------------------------------------------
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.

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] turboFei edited a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei edited a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-755054849


   The rc should be that:
   
   These two statements only return one result.
   
   Might the first statement contains an invalid statement `/* SELECT 'test';*/` and does not return result, or these two statements execute concurrently and conflict each other.
   
   ![image](https://user-images.githubusercontent.com/6757692/103725777-46a84b80-5012-11eb-9df9-dec105215259.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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-710014833






----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751484066


   **[Test build #133410 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133410/testReport)** for PR 29982 at commit [`f1befd3`](https://github.com/apache/spark/commit/f1befd3ab5cfcbfab1cbcd07b09396e4ecae5212).


----------------------------------------------------------------
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.

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] turboFei commented on a change in pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
turboFei commented on a change in pull request #29982:
URL: https://github.com/apache/spark/pull/29982#discussion_r551375840



##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
##########
@@ -573,4 +573,28 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
     // the date formatter for `java.sql.LocalDate` must output negative years with sign.
     runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15")
   }
+
+  test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
+    runCliWithin(1.minute)("/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)(";;/* SELECT 'test';*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/* SELECT 'test';*/;; SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("SELECT 'test'; -- SELECT 'test';" -> "")
+    runCliWithin(1.minute)("SELECT 'test'; /* SELECT 'test';*/" -> "")
+    runCliWithin(1.minute)("/*$meta chars{^\\;}*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*\nmulti-line\n*/ SELECT 'test';" -> "test")
+    runCliWithin(1.minute)("/*/* multi-level bracketed*/ SELECT 'test';" -> "test")
+  }
+
+  test("SPARK-33100: test sql statements with hint in bracketed comment") {
+    runCliWithin(2.minute)(
+      "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);"
+        -> "",
+      "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);"
+        -> "",
+      "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "SortMergeJoin",
+      "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;"
+        -> "BroadcastHashJoin"

Review comment:
       Seems ` /*+ MERGEJOIN(t1); */` does not conform the syntax.
   
   ```
   [info]   2021-01-04 07:09:03.961 - stdout> spark-sql> EXPLAIN SELECT /*+ MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;
   [info]   2021-01-04 07:09:03.967 - stderr> Error in query: 
   [info]   2021-01-04 07:09:03.967 - stderr> extraneous input ';' expecting {',', '*/', 'ADD', 'AFTER', 'ALL', 'ALTER', 'ANALYZE', 'AND', 'ANTI', 'ANY', 'ARCHIVE', 'ARRAY', 'AS', 'ASC', 'AT', 'AUTHORIZATION', 'BETWEEN', 'BOTH', 'BUCKET', 'BUCKETS', 'BY', 'CACHE', 'CASCADE', 'CASE', 'CAST', 'CHANGE', 'CHECK', 'CLEAR', 'CLUSTER', 'CLUSTERED', 'CODEGEN', 'COLLATE', 'COLLECTION', 'COLUMN', 'COLUMNS', 'COMMENT', 'COMMIT', 'COMPACT', 'COMPACTIONS', 'COMPUTE', 'CONCATENATE', 'CONSTRAINT', 'COST', 'CREATE', 'CROSS', 'CUBE', 'CURRENT', 'CURRENT_DATE', 'CURRENT_TIME', 'CURRENT_TIMESTAMP', 'CURRENT_USER', 'DATA', 'DATABASE', DATABASES, 'DBPROPERTIES', 'DEFINED', 'DELETE', 'DELIMITED', 'DESC', 'DESCRIBE', 'DFS', 'DIRECTORIES', 'DIRECTORY', 'DISTINCT', 'DISTRIBUTE', 'DIV', 'DROP', 'ELSE', 'END', 'ESCAPE', 'ESCAPED', 'EXCEPT', 'EXCHANGE', 'EXISTS', 'EXPLAIN', 'EXPORT', 'EXTENDED', 'EXTERNAL', 'EXTRACT', 'FALSE', 'FETCH', 'FIELDS', 'FILTER', 'FILEFORMAT', 'FIRST', 'FOLLOWING', 'FOR', 'FOREIGN', 'F
 ORMAT', 'FORMATTED', 'FROM', 'FULL', 'FUNCTION', 'FUNCTIONS', 'GLOBAL', 'GRANT', 'GROUP', 'GROUPING', 'HAVING', 'IF', 'IGNORE', 'IMPORT', 'IN', 'INDEX', 'INDEXES', 'INNER', 'INPATH', 'INPUTFORMAT', 'INSERT', 'INTERSECT', 'INTERVAL', 'INTO', 'IS', 'ITEMS', 'JOIN', 'KEYS', 'LAST', 'LATERAL', 'LAZY', 'LEADING', 'LEFT', 'LIKE', 'LIMIT', 'LINES', 'LIST', 'LOAD', 'LOCAL', 'LOCATION', 'LOCK', 'LOCKS', 'LOGICAL', 'MACRO', 'MAP', 'MATCHED', 'MERGE', 'MSCK', 'NAMESPACE', 'NAMESPACES', 'NATURAL', 'NO', NOT, 'NULL', 'NULLS', 'OF', 'ON', 'ONLY', 'OPTION', 'OPTIONS', 'OR', 'ORDER', 'OUT', 'OUTER', 'OUTPUTFORMAT', 'OVER', 'OVERLAPS', 'OVERLAY', 'OVERWRITE', 'PARTITION', 'PARTITIONED', 'PARTITIONS', 'PERCENT', 'PIVOT', 'PLACING', 'POSITION', 'PRECEDING', 'PRIMARY', 'PRINCIPALS', 'PROPERTIES', 'PURGE', 'QUERY', 'RANGE', 'RECORDREADER', 'RECORDWRITER', 'RECOVER', 'REDUCE', 'REFERENCES', 'REFRESH', 'RENAME', 'REPAIR', 'REPLACE', 'RESET', 'RESTRICT', 'REVOKE', 'RIGHT', RLIKE, 'ROLE', 'ROLES', 'ROLLBACK
 ', 'ROLLUP', 'ROW', 'ROWS', 'SCHEMA', 'SELECT', 'SEMI', 'SEPARATED', 'SERDE', 'SERDEPROPERTIES', 'SESSION_USER', 'SET', 'MINUS', 'SETS', 'SHOW', 'SKEWED', 'SOME', 'SORT', 'SORTED', 'START', 'STATISTICS', 'STORED', 'STRATIFY', 'STRUCT', 'SUBSTR', 'SUBSTRING', 'TABLE', 'TABLES', 'TABLESAMPLE', 'TBLPROPERTIES', TEMPORARY, 'TERMINATED', 'THEN', 'TIME', 'TO', 'TOUCH', 'TRAILING', 'TRANSACTION', 'TRANSACTIONS', 'TRANSFORM', 'TRIM', 'TRUE', 'TRUNCATE', 'TYPE', 'UNARCHIVE', 'UNBOUNDED', 'UNCACHE', 'UNION', 'UNIQUE', 'UNKNOWN', 'UNLOCK', 'UNSET', 'UPDATE', 'USE', 'USER', 'USING', 'VALUES', 'VIEW', 'VIEWS', 'WHEN', 'WHERE', 'WINDOW', 'WITH', 'ZONE', IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 32)
   [info]   2021-01-04 07:09:03.967 - stderr> 
   [info]   2021-01-04 07:09:03.967 - stderr> == SQL ==
   [info]   2021-01-04 07:09:03.967 - stderr> EXPLAIN SELECT /*+ MERGEJOIN(t1); */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v
   [info]   2021-01-04 07:09:03.967 - stderr> --------------------------------^^^
   [info]   2021-01-04 07:09:03.968 - stderr> 
   [info]   2021-01-04 07:09:04.102 - stdout> spark-sql> 
   
   ```




----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-750000855


   **[Test build #133275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133275/testReport)** for PR 29982 at commit [`5cbd308`](https://github.com/apache/spark/commit/5cbd30850b1403e34e489ba1adbbafc8cf82894a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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] AmplabJenkins commented on pull request #29982: [SPARK-33100][SQL] Support parse the sql statements with C-style comments

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-706017122






----------------------------------------------------------------
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.

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] AmplabJenkins removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-709335995


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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] SparkQA removed a comment on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-751491341


   **[Test build #133411 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133411/testReport)** for PR 29982 at commit [`55aa0c3`](https://github.com/apache/spark/commit/55aa0c3beb989933437ab81b5e10e9564f8046d2).


----------------------------------------------------------------
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.

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] SparkQA commented on pull request #29982: [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29982:
URL: https://github.com/apache/spark/pull/29982#issuecomment-753482541


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/38182/
   


----------------------------------------------------------------
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.

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