Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1399#discussion_r15155042
  
    --- Diff: 
sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
 ---
    @@ -0,0 +1,345 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.hive.thriftserver
    +
    +import scala.collection.JavaConversions._
    +
    +import java.io._
    +import java.util.{ArrayList => JArrayList}
    +
    +import jline.{ConsoleReader, History}
    +import org.apache.commons.lang.StringUtils
    +import org.apache.commons.logging.LogFactory
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, 
OptionsProcessor}
    +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException
    +import org.apache.hadoop.hive.common.{HiveInterruptCallback, 
HiveInterruptUtils, LogUtils}
    +import org.apache.hadoop.hive.conf.HiveConf
    +import org.apache.hadoop.hive.ql.Driver
    +import org.apache.hadoop.hive.ql.exec.Utilities
    +import org.apache.hadoop.hive.ql.processors.{CommandProcessor, 
CommandProcessorFactory}
    +import org.apache.hadoop.hive.ql.session.SessionState
    +import org.apache.hadoop.hive.shims.ShimLoader
    +import org.apache.thrift.transport.TSocket
    +
    +import org.apache.spark.sql.Logging
    +
    +object SparkSQLCLIDriver {
    +  private var prompt = "spark-sql"
    +  private var continuedPrompt = "".padTo(prompt.length, ' ')
    +  private var transport:TSocket = _
    +
    +  installSignalHandler()
    +
    +  /**
    +   * Install an interrupt callback to cancel all Spark jobs. In Hive's 
CliDriver#processLine(),
    +   * a signal handler will invoke this registered callback if a Ctrl+C 
signal is detected while
    +   * a command is being processed by the current thread.
    +   */
    +  def installSignalHandler() {
    +    HiveInterruptUtils.add(new HiveInterruptCallback {
    +      override def interrupt() {
    +        // Handle remote execution mode
    +        if (SparkSQLEnv.sparkContext != null) {
    +          SparkSQLEnv.sparkContext.cancelAllJobs()
    +        } else {
    +          if (transport != null) {
    +            // Force closing of TCP connection upon session termination
    +            transport.getSocket.close()
    +          }
    +        }
    +      }
    +    })
    +  }
    +
    +  def main(args: Array[String]) {
    +    val oproc = new OptionsProcessor()
    +    if (!oproc.process_stage1(args)) {
    +      System.exit(1)
    +    }
    +
    +    // NOTE: It is critical to do this here so that log4j is reinitialized
    +    // before any of the other core hive classes are loaded
    +    var logInitFailed = false
    +    var logInitDetailMessage: String = null
    +    try {
    +      logInitDetailMessage = LogUtils.initHiveLog4j()
    +    } catch {
    +      case e: LogInitializationException =>
    +        logInitFailed = true
    +        logInitDetailMessage = e.getMessage
    +    }
    +
    +    val sessionState = new CliSessionState(new 
HiveConf(classOf[SessionState]))
    +
    +    sessionState.in = System.in
    +    try {
    +      sessionState.out = new PrintStream(System.out, true, "UTF-8")
    +      sessionState.info = new PrintStream(System.err, true, "UTF-8")
    +      sessionState.err = new PrintStream(System.err, true, "UTF-8")
    +    } catch {
    +      case e: UnsupportedEncodingException => System.exit(3)
    +    }
    +
    +    if (!oproc.process_stage2(sessionState)) {
    +      System.exit(2)
    +    }
    +
    +    if (!sessionState.getIsSilent) {
    +      if (logInitFailed) System.err.println(logInitDetailMessage)
    +      else SessionState.getConsole.printInfo(logInitDetailMessage)
    +    }
    +
    +    // Set all properties specified via command line.
    +    val conf: HiveConf = sessionState.getConf
    +    sessionState.cmdProperties.entrySet().foreach { item: 
java.util.Map.Entry[Object, Object] =>
    +      conf.set(item.getKey.asInstanceOf[String], 
item.getValue.asInstanceOf[String])
    +      sessionState.getOverriddenConfigurations.put(
    +        item.getKey.asInstanceOf[String], 
item.getValue.asInstanceOf[String])
    +    }
    +
    +    SessionState.start(sessionState)
    +
    +    // Clean up after we exit
    +    Runtime.getRuntime.addShutdownHook(
    +      new Thread() {
    +        override def run() {
    +          SparkSQLEnv.stop()
    +        }
    +      }
    +    )
    +
    +    // "-h" option has been passed, so connect to Hive thrift server.
    +    if (sessionState.getHost != null) {
    +      sessionState.connect()
    +      if (sessionState.isRemoteMode) {
    +        prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + 
prompt
    +        continuedPrompt = "".padTo(prompt.length, ' ')
    +      }
    +    }
    +
    +    if (!sessionState.isRemoteMode && 
!ShimLoader.getHadoopShims.usesJobShell()) {
    +      // Hadoop-20 and above - we need to augment classpath using hiveconf
    +      // components.
    +      // See also: code in ExecDriver.java
    +      var loader = conf.getClassLoader
    +      val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS)
    +      if (StringUtils.isNotBlank(auxJars)) {
    +        loader = Utilities.addToClassPath(loader, 
StringUtils.split(auxJars, ","))
    +      }
    +      conf.setClassLoader(loader)
    +      Thread.currentThread().setContextClassLoader(loader)
    +    }
    +
    +    val cli = new SparkSQLCLIDriver
    +    cli.setHiveVariables(oproc.getHiveVariables)
    +
    +    // TODO work around for set the log output to console, because the 
HiveContext
    +    // will set the output into an invalid buffer.
    +    sessionState.in = System.in
    +    try {
    +      sessionState.out = new PrintStream(System.out, true, "UTF-8")
    +      sessionState.info = new PrintStream(System.err, true, "UTF-8")
    +      sessionState.err = new PrintStream(System.err, true, "UTF-8")
    +    } catch {
    +      case e: UnsupportedEncodingException => System.exit(3)
    +    }
    +
    +    // Execute -i init files (always in silent mode)
    +    cli.processInitFiles(sessionState)
    +
    +    if (sessionState.execString != null) {
    +      System.exit(cli.processLine(sessionState.execString))
    +    }
    +
    +    try {
    +      if (sessionState.fileName != null) {
    +        System.exit(cli.processFile(sessionState.fileName))
    +      }
    +    } catch {
    +      case e: FileNotFoundException =>
    +        System.err.println(s"Could not open input file for reading. 
(${e.getMessage})")
    +        System.exit(3)
    +    }
    +
    +    val reader = new ConsoleReader()
    +    reader.setBellEnabled(false)
    +    // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", 
true)))
    +    CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e))
    +
    +    val historyDirectory = System.getProperty("user.home")
    +
    +    try {
    +      if (new File(historyDirectory).exists()) {
    +        val historyFile = historyDirectory + File.separator + 
".hivehistory"
    +        reader.setHistory(new History(new File(historyFile)))
    +      } else {
    +        System.err.println("WARNING: Directory for Hive history file: " + 
historyDirectory +
    +                           " does not exist.   History will not be 
available during this session.")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        System.err.println("WARNING: Encountered an error while trying to 
initialize Hive's " +
    +                           "history file.  History will not be available 
during this session.")
    +        System.err.println(e.getMessage)
    +    }
    +
    +    val clientTransportTSocketField = 
classOf[CliSessionState].getDeclaredField("transport")
    +    clientTransportTSocketField.setAccessible(true)
    +
    +    transport = 
clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket]
    +
    +    var ret = 0
    +    var prefix = ""
    +    val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], 
"getFormattedDb",
    +      classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState)
    +
    +    def promptWithCurrentDB = s"$prompt$currentDB"
    +    def continuedPromptWithDBSpaces = continuedPrompt + 
ReflectionUtils.invokeStatic(
    +      classOf[CliDriver], "spacesForString", classOf[String] -> currentDB)
    +
    +    var currentPrompt = promptWithCurrentDB
    +    var line = reader.readLine(currentPrompt + "> ")
    +
    +    while (line != null) {
    +      if (prefix.nonEmpty) {
    +        prefix += '\n'
    +      }
    +
    +      if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) {
    +        line = prefix + line
    +        ret = cli.processLine(line, true)
    +        prefix = ""
    +        currentPrompt = promptWithCurrentDB
    +      } else {
    +        prefix = prefix + line
    +        currentPrompt = continuedPromptWithDBSpaces
    +      }
    +
    +      line = reader.readLine(currentPrompt + "> ")
    +    }
    +
    +    sessionState.close()
    +
    +    System.exit(ret)
    +  }
    +}
    +
    +
    +class SparkSQLCLIDriver extends CliDriver with Logging {
    --- End diff --
    
    Any reason not to make all the classes here private or private[spark]?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to