wangyum commented on code in PR #40838:
URL: https://github.com/apache/spark/pull/40838#discussion_r1173322839
##########
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:
1. It is not only for `set x=y`. Sometimes we also want to get the value.
e.g.: `set spark.sql.warehouse.dir;`.
2. `Completer` will sorted it automatically.
```
spark-sql (default)> Set spark.sql.
Display all 200 possibilities? (y or n)
spark.sql.adaptive.advisoryPartitionSizeInBytes
spark.sql.adaptive.autoBroadcastJoinThreshold
spark.sql.adaptive.coalescePartitions.enabled
spark.sql.adaptive.coalescePartitions.initialPartitionNum
spark.sql.adaptive.coalescePartitions.minPartitionSize
spark.sql.adaptive.coalescePartitions.parallelismFirst
spark.sql.adaptive.customCostEvaluatorClass
spark.sql.adaptive.enabled
spark.sql.adaptive.forceOptimizeSkewedJoin
spark.sql.adaptive.localShuffleReader.enabled
spark.sql.adaptive.maxShuffledHashJoinLocalMapThreshold
spark.sql.adaptive.optimizeSkewsInRebalancePartitions.enabled
spark.sql.adaptive.optimizer.excludedRules
spark.sql.adaptive.rebalancePartitionsSmallPartitionFactor
spark.sql.adaptive.skewJoin.enabled
spark.sql.adaptive.skewJoin.skewedPartitionFactor
spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes
spark.sql.ansi.doubleQuotedIdentifiers
spark.sql.ansi.enabled
spark.sql.ansi.enforceReservedKeywords
spark.sql.ansi.relationPrecedence
spark.sql.autoBroadcastJoinThreshold
spark.sql.avro.compression.codec
spark.sql.avro.deflate.level
spark.sql.avro.filterPushdown.enabled
spark.sql.broadcastTimeout
spark.sql.bucketing.coalesceBucketsInJoin.enabled
spark.sql.bucketing.coalesceBucketsInJoin.maxBucketRatio
spark.sql.cache.serializer
spark.sql.catalog.spark_catalog
spark.sql.catalog.spark_catalog.defaultDatabase
spark.sql.cbo.enabled
spark.sql.cbo.joinReorder.dp.star.filter
spark.sql.cbo.joinReorder.dp.threshold
spark.sql.cbo.joinReorder.enabled
spark.sql.cbo.planStats.enabled
spark.sql.cbo.starSchemaDetection
spark.sql.charAsVarchar
spark.sql.cli.print.header
spark.sql.columnNameOfCorruptRecord
spark.sql.csv.filterPushdown.enabled
spark.sql.datetime.java8API.enabled
spark.sql.debug.maxToStringFields
spark.sql.defaultCatalog
spark.sql.error.messageFormat
spark.sql.event.truncate.length
spark.sql.execution.arrow.enabled
spark.sql.execution.arrow.fallback.enabled
spark.sql.execution.arrow.localRelationThreshold
spark.sql.execution.arrow.maxRecordsPerBatch
spark.sql.execution.arrow.pyspark.enabled
spark.sql.execution.arrow.pyspark.fallback.enabled
spark.sql.execution.arrow.pyspark.selfDestruct.enabled
spark.sql.execution.arrow.sparkr.enabled
spark.sql.execution.pandas.udf.buffer.size
spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled
spark.sql.execution.pythonUDF.arrow.enabled
spark.sql.execution.topKSortFallbackThreshold
spark.sql.extensions
spark.sql.files.ignoreCorruptFiles
spark.sql.files.ignoreMissingFiles
spark.sql.files.maxPartitionBytes
spark.sql.files.maxRecordsPerFile
spark.sql.files.minPartitionNum
spark.sql.function.concatBinaryAsString
spark.sql.function.eltOutputAsString
spark.sql.groupByAliases
spark.sql.groupByOrdinal
spark.sql.hive.convertInsertingPartitionedTable
spark.sql.hive.convertMetastoreCtas
spark.sql.hive.convertMetastoreInsertDir
spark.sql.hive.convertMetastoreOrc
spark.sql.hive.convertMetastoreParquet
spark.sql.hive.convertMetastoreParquet.mergeSchema
spark.sql.hive.dropPartitionByName.enabled
spark.sql.hive.filesourcePartitionFileCacheSize
spark.sql.hive.manageFilesourcePartitions
spark.sql.hive.metastore.barrierPrefixes
spark.sql.hive.metastore.jars
spark.sql.hive.metastore.jars.path
spark.sql.hive.metastore.sharedPrefixes
spark.sql.hive.metastore.version
spark.sql.hive.metastorePartitionPruning
spark.sql.hive.metastorePartitionPruningFallbackOnException
spark.sql.hive.metastorePartitionPruningFastFallback
spark.sql.hive.thriftServer.async
spark.sql.hive.thriftServer.singleSession
spark.sql.hive.verifyPartitionPath
spark.sql.hive.version
spark.sql.inMemoryColumnarStorage.batchSize
spark.sql.inMemoryColumnarStorage.compressed
spark.sql.inMemoryColumnarStorage.enableVectorizedReader
spark.sql.json.filterPushdown.enabled
spark.sql.jsonGenerator.ignoreNullFields
spark.sql.leafNodeDefaultParallelism
spark.sql.mapKeyDedupPolicy
spark.sql.maven.additionalRemoteRepositories
spark.sql.maxMetadataStringLength
spark.sql.maxPlanStringLength
spark.sql.maxSinglePartitionBytes
spark.sql.metadataCacheTTLSeconds
spark.sql.optimizer.collapseProjectAlwaysInline
spark.sql.optimizer.dynamicPartitionPruning.enabled
spark.sql.optimizer.enableCsvExpressionOptimization
spark.sql.optimizer.enableJsonExpressionOptimization
spark.sql.optimizer.excludedRules
spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold
spark.sql.optimizer.runtime.bloomFilter.creationSideThreshold
spark.sql.optimizer.runtime.bloomFilter.enabled
spark.sql.optimizer.runtime.bloomFilter.expectedNumItems
spark.sql.optimizer.runtime.bloomFilter.maxNumBits
spark.sql.optimizer.runtime.bloomFilter.maxNumItems
spark.sql.optimizer.runtime.bloomFilter.numBits
spark.sql.optimizer.runtime.rowLevelOperationGroupFilter.enabled
spark.sql.optimizer.runtimeFilter.number.threshold
spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled
spark.sql.orc.aggregatePushdown
spark.sql.orc.columnarReaderBatchSize
spark.sql.orc.columnarWriterBatchSize
spark.sql.orc.compression.codec
spark.sql.orc.enableNestedColumnVectorizedReader
spark.sql.orc.enableVectorizedReader
spark.sql.orc.filterPushdown
spark.sql.orc.mergeSchema
spark.sql.orderByOrdinal
spark.sql.parquet.aggregatePushdown
spark.sql.parquet.binaryAsString
spark.sql.parquet.columnarReaderBatchSize
spark.sql.parquet.compression.codec
spark.sql.parquet.enableNestedColumnVectorizedReader
spark.sql.parquet.enableVectorizedReader
spark.sql.parquet.fieldId.read.enabled
spark.sql.parquet.fieldId.read.ignoreMissing
spark.sql.parquet.fieldId.write.enabled
spark.sql.parquet.filterPushdown
spark.sql.parquet.inferTimestampNTZ.enabled
spark.sql.parquet.int96AsTimestamp
spark.sql.parquet.int96TimestampConversion
spark.sql.parquet.mergeSchema
spark.sql.parquet.outputTimestampType
spark.sql.parquet.recordLevelFilter.enabled
spark.sql.parquet.respectSummaryFiles
spark.sql.parquet.writeLegacyFormat
spark.sql.parser.quotedRegexColumnNames
spark.sql.pivotMaxValues
spark.sql.pyspark.inferNestedDictAsStruct.enabled
spark.sql.pyspark.jvmStacktrace.enabled
spark.sql.pyspark.legacy.inferArrayTypeFromFirstElement.enabled
spark.sql.queryExecutionListeners
spark.sql.readSideCharPadding
spark.sql.redaction.options.regex
spark.sql.redaction.string.regex
spark.sql.repl.eagerEval.enabled
spark.sql.repl.eagerEval.maxNumRows
spark.sql.repl.eagerEval.truncate
spark.sql.session.timeZone
spark.sql.shuffle.partitions
spark.sql.shuffledHashJoinFactor
spark.sql.sources.bucketing.autoBucketedScan.enabled
spark.sql.sources.bucketing.enabled
spark.sql.sources.bucketing.maxBuckets
spark.sql.sources.default
spark.sql.sources.disabledJdbcConnProviderList
spark.sql.sources.parallelPartitionDiscovery.threshold
spark.sql.sources.partitionColumnTypeInference.enabled
spark.sql.sources.partitionOverwriteMode
spark.sql.sources.v2.bucketing.enabled
spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled
spark.sql.sources.v2.bucketing.pushPartValues.enabled
spark.sql.statistics.fallBackToHdfs
spark.sql.statistics.histogram.enabled
spark.sql.statistics.size.autoUpdate.enabled
spark.sql.storeAssignmentPolicy
spark.sql.streaming.checkpointLocation
spark.sql.streaming.continuous.epochBacklogQueueSize
spark.sql.streaming.disabledV2Writers
spark.sql.streaming.fileSource.cleaner.numThreads
spark.sql.streaming.forceDeleteTempCheckpointLocation
spark.sql.streaming.metricsEnabled
spark.sql.streaming.multipleWatermarkPolicy
spark.sql.streaming.noDataMicroBatches.enabled
spark.sql.streaming.numRecentProgressUpdates
spark.sql.streaming.sessionWindow.merge.sessions.in.local.partition
spark.sql.streaming.stateStore.stateSchemaCheck
spark.sql.streaming.stopActiveRunOnRestart
spark.sql.streaming.stopTimeout
spark.sql.streaming.streamingQueryListeners
spark.sql.streaming.ui.enabled
spark.sql.streaming.ui.retainedProgressUpdates
spark.sql.streaming.ui.retainedQueries
spark.sql.thriftServer.interruptOnCancel
spark.sql.thriftServer.queryTimeout
spark.sql.thriftserver.scheduler.pool
spark.sql.thriftserver.ui.retainedSessions
spark.sql.thriftserver.ui.retainedStatements
spark.sql.timestampType
spark.sql.ui.explainMode
spark.sql.ui.retainedExecutions
spark.sql.variable.substitute
spark.sql.warehouse.dir
```
--
This is an automated message from the 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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]