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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1075,7 +1075,7 @@ fragment LETTER
;

SIMPLE_COMMENT
: '--' ~[\r\n]* '\r'? '\n'? -> channel(HIDDEN)
: '--' ('\\\n' | ~[\r\n])* '\r'? '\n'? -> channel(HIDDEN)
;

BRACKETED_EMPTY_COMMENT
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,16 @@ class PlanParserSuite extends AnalysisTest {
}
}

test("single comment case one") {
val plan = table("a").select(star())
assertEqual("-- single comment\nSELECT * FROM a", plan)
}

test("single comment case two") {
val plan = table("a").select(star())
assertEqual("-- single comment\\\nwith line continuity\nSELECT * FROM a", plan)
}

test("case insensitive") {
val plan = table("a").select(star())
assertEqual("sELEct * FroM a", plan)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.hive.thriftserver

import java.io._
import java.util.{ArrayList => JArrayList, Locale}
import java.util.{ArrayList => JArrayList, List => JList, Locale}

import scala.collection.JavaConverters._

Expand All @@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.apache.log4j.Level
import org.apache.thrift.transport.TSocket
import sun.misc.{Signal, SignalHandler}

import org.apache.spark.SparkConf
import org.apache.spark.deploy.SparkHadoopUtil
Expand Down Expand Up @@ -434,5 +435,157 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
ret
}
}

// Adapted processLine from Hive 2.3's CliDriver.processLine.
override def processLine(line: String, allowInterrupting: Boolean): Int = {
var oldSignal: SignalHandler = null
var interruptSignal: Signal = null

if (allowInterrupting) {
// Remember all threads that were running at the time we started line processing.
// Hook up the custom Ctrl+C handler while processing this line
interruptSignal = new Signal("INT")
oldSignal = Signal.handle(interruptSignal, new SignalHandler() {
private var interruptRequested: Boolean = false

override def handle(signal: Signal) {
val initialRequest = !interruptRequested
interruptRequested = true

// Kill the VM on second ctrl+c
if (!initialRequest) {
console.printInfo("Exiting the JVM")
System.exit(127)
}

// Interrupt the CLI thread to stop the current statement and return
// to prompt
console.printInfo("Interrupting... Be patient, this might take some time.")
console.printInfo("Press Ctrl+C again to kill JVM")

HiveInterruptUtils.interrupt()
}
})
}

try {
var lastRet: Int = 0

// we can not use "split" function directly as ";" may be quoted
val commands = splitSemiColon(line).asScala
var command: String = ""
for (oneCmd <- commands) {
if (StringUtils.endsWith(oneCmd, "\\")) {
command += StringUtils.chop(oneCmd) + ";"
} else {
command += oneCmd
if (!StringUtils.isBlank(command)) {
val ret = processCmd(command)
command = ""
lastRet = ret
val ignoreErrors = HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS)
if (ret != 0 && !ignoreErrors) {
CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf])
ret
}
}
}
}
CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf])
lastRet
} finally {
// Once we are done processing the line, restore the old handler
if (oldSignal != null && interruptSignal != null) {
Signal.handle(interruptSignal, oldSignal)
}
}
}

// Adapted splitSemiColon from Hive 2.3's CliDriver.splitSemiColon.
// 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 insideSimpleComment = false
var bracketedCommentLevel = 0
var escape = false
var beginIndex = 0
var includingStatement = false
val ret = new JArrayList[String]

def insideBracketedComment: Boolean = bracketedCommentLevel > 0
def insideComment: Boolean = insideSimpleComment || insideBracketedComment
def statementBegin(index: Int): Boolean = includingStatement || (!insideComment &&
index > beginIndex && !s"${line.charAt(index)}".trim.isEmpty)

for (index <- 0 until line.length) {
if (line.charAt(index) == '\'' && !insideComment) {
// take a look to see if it is escaped
// See the comment above about SPARK-31595
if (!escape && !insideDoubleQuote) {
// flip the boolean variable
insideSingleQuote = !insideSingleQuote
}
} else if (line.charAt(index) == '\"' && !insideComment) {
// take a look to see if it is escaped
// See the comment above about SPARK-31595
if (!escape && !insideSingleQuote) {
// flip the boolean variable
insideDoubleQuote = !insideDoubleQuote
}
} else if (line.charAt(index) == '-') {
val hasNext = index + 1 < line.length
if (insideDoubleQuote || insideSingleQuote || insideComment) {
// Ignores '-' in any case of quotes or comment.
// Avoids to start a comment(--) within a quoted segment or already in a comment.
// Sample query: select "quoted value --"
// ^^ avoids starting a comment if it's inside quotes.
} else if (hasNext && line.charAt(index + 1) == '-') {
// ignore quotes and ; in simple comment
insideSimpleComment = true
}
} else if (line.charAt(index) == ';') {
if (insideSingleQuote || insideDoubleQuote || insideComment) {
// do not split
} else {
if (includingStatement) {
// split, do not include ; itself
ret.add(line.substring(beginIndex, index))
}
beginIndex = index + 1
includingStatement = false
}
} else if (line.charAt(index) == '\n') {
// with a new line the inline simple comment should end.
if (!escape) {
insideSimpleComment = false
}
} else if (line.charAt(index) == '/' && !insideSimpleComment) {
val hasNext = index + 1 < line.length
if (insideSingleQuote || insideDoubleQuote) {
// Ignores '/' in any case of quotes
} else if (insideBracketedComment && line.charAt(index - 1) == '*' ) {
bracketedCommentLevel -= 1
} else if (hasNext && !insideBracketedComment && line.charAt(index + 1) == '*') {
bracketedCommentLevel += 1
}
}
// set the escape
if (escape) {
escape = false
} else if (line.charAt(index) == '\\') {
escape = true
}

includingStatement = statementBegin(index)
}
if (includingStatement) {
ret.add(line.substring(beginIndex))
}
ret
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -296,4 +296,67 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
"set spark.sql.warehouse.dir;" -> tmpDir.getAbsolutePath)
tmpDir.delete()
}

test("SPARK-26321 Should not split semicolon within quoted string literals") {
runCliWithin(3.minute)(
"""select 'Test1', "^;^";""" -> "Test1\t^;^",
"""select 'Test2', "\";";""" -> "Test2\t\";",
"""select 'Test3', "\';";""" -> "Test3\t';",
"select concat('Test4', ';');" -> "Test4;"
)
}

test("SPARK-30049 Should not complain for quotes in commented lines") {
runCliWithin(1.minute)(
"""SELECT concat('test', 'comment') -- someone's comment here
|;""".stripMargin -> "testcomment"
)
}

test("SPARK-31102 spark-sql fails to parse when contains comment") {
runCliWithin(1.minute)(
"""SELECT concat('test', 'comment'),
| -- someone's comment here
| 2;""".stripMargin -> "testcomment"
)
}

test("SPARK-30049 Should not complain for quotes in commented with multi-lines") {
runCliWithin(1.minute)(
"""SELECT concat('test', 'comment') -- someone's comment here \
| comment continues here with single ' quote \
| extra ' \
|;""".stripMargin -> "testcomment"
)
}

test("SPARK-31595 Should allow unescaped quote mark in quoted string") {
runCliWithin(1.minute)(
"SELECT '\"legal string a';select 1 + 234;".stripMargin -> "235"
)
runCliWithin(1.minute)(
"SELECT \"legal 'string b\";select 22222 + 1;".stripMargin -> "22223"
)
}

test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") {
runCliWithin(4.minute)(
"/* SELECT 'test';*/ SELECT 'test';" -> "test",
";;/* SELECT 'test';*/ SELECT 'test';" -> "test",
"/* SELECT 'test';*/;; SELECT 'test';" -> "test",
"SELECT 'test'; -- SELECT 'test';" -> "",
"SELECT 'test'; /* SELECT 'test';*/;" -> "",
"/*$meta chars{^\\;}*/ SELECT 'test';" -> "test",
"/*\nmulti-line\n*/ SELECT 'test';" -> "test",
"/*/* multi-level bracketed*/ SELECT 'test';" -> "test"
)
}

test("SPARK-33100: test sql statements with hint in bracketed comment") {
runCliWithin(2.minute)(
"CREATE TEMPORARY VIEW t AS SELECT * FROM VALUES(1, 2) AS t(k, v);" -> "",
"EXPLAIN EXTENDED SELECT /*+ broadcast(t) */ * from t;"
-> "ResolvedHint (strategy=broadcast)"
Copy link
Member Author

Choose a reason for hiding this comment

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

because spark-2.4 does not support mergeJoin hint, so I modify this ut

)
}
}