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

[GitHub] [spark] yaooqinn commented on a diff in pull request #40838: [SPARK-43174][SQL] Fix SparkSQLCLIDriver completer

yaooqinn commented on code in PR #40838:
URL: https://github.com/apache/spark/pull/40838#discussion_r1173270795


##########
sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala:
##########
@@ -336,6 +338,86 @@ private[hive] object SparkSQLCLIDriver extends Logging {
       state.close()
     }
   }
+
+  private def getCommandCompleter(context: SQLContext): Array[Completer] = {
+    // StringsCompleter matches against a pre-defined wordlist
+    // We start with an empty wordlist and build it up
+    val candidateStrings = new JArrayList[String]
+    // We add Spark SQL function names
+    // For functions that aren't infix operators, we add an open
+    // parenthesis at the end.
+    context.sessionState.functionRegistry.listFunction().map(_.funcName).foreach { s =>
+      if (s.matches("[a-z_]+")) {
+        candidateStrings.add(s + "(")
+      } else {
+        candidateStrings.add(s)
+      }
+    }
+    // We add Spark SQL keywords, including lower-cased versions
+    SQLKeywordUtils.keywords.foreach { s =>
+      candidateStrings.add(s)
+      candidateStrings.add(s.toLowerCase(Locale.ROOT))
+    }
+
+    val strCompleter = new StringsCompleter(candidateStrings)
+    // Because we use parentheses in addition to whitespace
+    // as a keyword delimiter, we need to define a new ArgumentDelimiter
+    // that recognizes parenthesis as a delimiter.
+    val delim = new ArgumentCompleter.AbstractArgumentDelimiter() {
+      override def isDelimiterChar(buffer: CharSequence, pos: Int): Boolean = {
+        val c = buffer.charAt(pos)
+        Character.isWhitespace(c) || c == '(' || c == ')' || c == '[' || c == ']'
+      }
+    }
+    // The ArgumentCompleter allows us to match multiple tokens
+    // in the same line.
+    val argCompleter = new ArgumentCompleter(delim, strCompleter)
+    // By default ArgumentCompleter is in "strict" mode meaning
+    // a token is only auto-completed if all prior tokens
+    // match. We don't want that since there are valid tokens
+    // that are not in our wordlist (eg. table and column names)
+    argCompleter.setStrict(false)
+    // ArgumentCompleter always adds a space after a matched token.
+    // This is undesirable for function names because a space after
+    // the opening parenthesis is unnecessary (and uncommon) in Hive.
+    // We stack a custom Completer on top of our ArgumentCompleter
+    // to reverse this.
+    val customCompleter: Completer = new Completer() {
+      override def complete(buffer: String, offset: Int, completions: JList[CharSequence]): Int = {
+        val comp: JList[String] = completions.asInstanceOf[JList[String]]
+        val ret = argCompleter.complete(buffer, offset, completions)
+        // ConsoleReader will do the substitution if and only if there
+        // is exactly one valid completion, so we ignore other cases.
+        if (completions.size == 1 && comp.get(0).endsWith("( ")) comp.set(0, comp.get(0).trim)
+        ret
+      }
+    }
+
+    val confCompleter = new StringsCompleter(context.conf.getAllDefinedConfs.map(_._1).asJava) {

Review Comment:
   The static configurations are better to be hidden. The list is better to be sorted



##########
sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala:
##########
@@ -336,6 +338,86 @@ private[hive] object SparkSQLCLIDriver extends Logging {
       state.close()
     }
   }
+
+  private def getCommandCompleter(context: SQLContext): Array[Completer] = {
+    // StringsCompleter matches against a pre-defined wordlist
+    // We start with an empty wordlist and build it up
+    val candidateStrings = new JArrayList[String]
+    // We add Spark SQL function names
+    // For functions that aren't infix operators, we add an open
+    // parenthesis at the end.
+    context.sessionState.functionRegistry.listFunction().map(_.funcName).foreach { s =>
+      if (s.matches("[a-z_]+")) {
+        candidateStrings.add(s + "(")
+      } else {
+        candidateStrings.add(s)
+      }
+    }
+    // We add Spark SQL keywords, including lower-cased versions
+    SQLKeywordUtils.keywords.foreach { s =>
+      candidateStrings.add(s)
+      candidateStrings.add(s.toLowerCase(Locale.ROOT))
+    }
+
+    val strCompleter = new StringsCompleter(candidateStrings)
+    // Because we use parentheses in addition to whitespace
+    // as a keyword delimiter, we need to define a new ArgumentDelimiter
+    // that recognizes parenthesis as a delimiter.
+    val delim = new ArgumentCompleter.AbstractArgumentDelimiter() {
+      override def isDelimiterChar(buffer: CharSequence, pos: Int): Boolean = {
+        val c = buffer.charAt(pos)
+        Character.isWhitespace(c) || c == '(' || c == ')' || c == '[' || c == ']'
+      }
+    }
+    // The ArgumentCompleter allows us to match multiple tokens
+    // in the same line.
+    val argCompleter = new ArgumentCompleter(delim, strCompleter)
+    // By default ArgumentCompleter is in "strict" mode meaning
+    // a token is only auto-completed if all prior tokens
+    // match. We don't want that since there are valid tokens
+    // that are not in our wordlist (eg. table and column names)
+    argCompleter.setStrict(false)
+    // ArgumentCompleter always adds a space after a matched token.
+    // This is undesirable for function names because a space after
+    // the opening parenthesis is unnecessary (and uncommon) in Hive.
+    // We stack a custom Completer on top of our ArgumentCompleter
+    // to reverse this.
+    val customCompleter: Completer = new Completer() {
+      override def complete(buffer: String, offset: Int, completions: JList[CharSequence]): Int = {
+        val comp: JList[String] = completions.asInstanceOf[JList[String]]
+        val ret = argCompleter.complete(buffer, offset, completions)
+        // ConsoleReader will do the substitution if and only if there
+        // is exactly one valid completion, so we ignore other cases.
+        if (completions.size == 1 && comp.get(0).endsWith("( ")) comp.set(0, comp.get(0).trim)
+        ret
+      }
+    }
+
+    val confCompleter = new StringsCompleter(context.conf.getAllDefinedConfs.map(_._1).asJava) {
+      override def complete(buffer: String, cursor: Int, clist: JList[CharSequence]): Int = {
+        super.complete(buffer, cursor, clist)
+      }
+    }
+
+    val setCompleter = new StringsCompleter("set") {
+      override def complete(buffer: String, cursor: Int, clist: JList[CharSequence]): Int = {
+        if (buffer != null && buffer.equals("set")) {
+          super.complete(buffer, cursor, clist)
+        } else {
+          -1
+        }
+      }
+    }
+
+    val propCompleter = new ArgumentCompleter(setCompleter, confCompleter) {

Review Comment:
   The hive conf seems to work both for prompting and execution. shall we keep them?



##########
sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala:
##########
@@ -336,6 +338,86 @@ private[hive] object SparkSQLCLIDriver extends Logging {
       state.close()
     }
   }
+
+  private def getCommandCompleter(context: SQLContext): Array[Completer] = {
+    // StringsCompleter matches against a pre-defined wordlist
+    // We start with an empty wordlist and build it up
+    val candidateStrings = new JArrayList[String]
+    // We add Spark SQL function names
+    // For functions that aren't infix operators, we add an open
+    // parenthesis at the end.
+    context.sessionState.functionRegistry.listFunction().map(_.funcName).foreach { s =>
+      if (s.matches("[a-z_]+")) {
+        candidateStrings.add(s + "(")
+      } else {
+        candidateStrings.add(s)
+      }
+    }
+    // We add Spark SQL keywords, including lower-cased versions
+    SQLKeywordUtils.keywords.foreach { s =>
+      candidateStrings.add(s)
+      candidateStrings.add(s.toLowerCase(Locale.ROOT))
+    }
+
+    val strCompleter = new StringsCompleter(candidateStrings)
+    // Because we use parentheses in addition to whitespace
+    // as a keyword delimiter, we need to define a new ArgumentDelimiter
+    // that recognizes parenthesis as a delimiter.
+    val delim = new ArgumentCompleter.AbstractArgumentDelimiter() {
+      override def isDelimiterChar(buffer: CharSequence, pos: Int): Boolean = {
+        val c = buffer.charAt(pos)
+        Character.isWhitespace(c) || c == '(' || c == ')' || c == '[' || c == ']'
+      }
+    }
+    // The ArgumentCompleter allows us to match multiple tokens
+    // in the same line.
+    val argCompleter = new ArgumentCompleter(delim, strCompleter)
+    // By default ArgumentCompleter is in "strict" mode meaning
+    // a token is only auto-completed if all prior tokens
+    // match. We don't want that since there are valid tokens
+    // that are not in our wordlist (eg. table and column names)
+    argCompleter.setStrict(false)
+    // ArgumentCompleter always adds a space after a matched token.
+    // This is undesirable for function names because a space after
+    // the opening parenthesis is unnecessary (and uncommon) in Hive.
+    // We stack a custom Completer on top of our ArgumentCompleter
+    // to reverse this.
+    val customCompleter: Completer = new Completer() {
+      override def complete(buffer: String, offset: Int, completions: JList[CharSequence]): Int = {
+        val comp: JList[String] = completions.asInstanceOf[JList[String]]
+        val ret = argCompleter.complete(buffer, offset, completions)
+        // ConsoleReader will do the substitution if and only if there
+        // is exactly one valid completion, so we ignore other cases.
+        if (completions.size == 1 && comp.get(0).endsWith("( ")) comp.set(0, comp.get(0).trim)
+        ret
+      }
+    }
+
+    val confCompleter = new StringsCompleter(context.conf.getAllDefinedConfs.map(_._1).asJava) {
+      override def complete(buffer: String, cursor: Int, clist: JList[CharSequence]): Int = {
+        super.complete(buffer, cursor, clist)
+      }
+    }
+
+    val setCompleter = new StringsCompleter("set") {
+      override def complete(buffer: String, cursor: Int, clist: JList[CharSequence]): Int = {
+        if (buffer != null && buffer.equals("set")) {

Review Comment:
   ignore 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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