Skip to content

Commit

Permalink
Revert changes related to SPARK-2678, decided to move them to another PR
Browse files Browse the repository at this point in the history
  • Loading branch information
liancheng committed Jul 25, 2014
1 parent 21c6cf4 commit 090beea
Show file tree
Hide file tree
Showing 9 changed files with 67 additions and 54 deletions.
23 changes: 22 additions & 1 deletion bin/beeline
Original file line number Diff line number Diff line change
Expand Up @@ -20,5 +20,26 @@
# Figure out where Spark is installed
FWDIR="$(cd `dirname $0`/..; pwd)"

# Find the java binary
if [ -n "${JAVA_HOME}" ]; then
RUNNER="${JAVA_HOME}/bin/java"
else
if [ `command -v java` ]; then
RUNNER="java"
else
echo "JAVA_HOME is not set" >&2
exit 1
fi
fi

# Compute classpath using external script
classpath_output=$($FWDIR/bin/compute-classpath.sh)
if [[ "$?" != "0" ]]; then
echo "$classpath_output"
exit 1
else
CLASSPATH=$classpath_output
fi

CLASS="org.apache.hive.beeline.BeeLine"
exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal "$@"
exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@"
5 changes: 4 additions & 1 deletion bin/spark-sql
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,9 @@
#
# Shell script for starting the Spark SQL CLI

# Enter posix mode for bash
set -o posix

# Figure out where Spark is installed
FWDIR="$(cd `dirname $0`/..; pwd)"

Expand All @@ -30,4 +33,4 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
fi

CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver"
exec "$FWDIR"/bin/spark-submit --class $CLASS $@ spark-internal
exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {

/** Fill in values by parsing user options. */
private def parseOpts(opts: Seq[String]): Unit = {
val EQ_SEPARATED_OPT = """(--[^=]+)=(.+)""".r
var inSparkOpts = true

// Delineates parsing of Spark options from parsing of user options.
parse(opts)
Expand Down Expand Up @@ -307,21 +307,33 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
verbose = true
parse(tail)

case EQ_SEPARATED_OPT(opt, value) :: tail =>
// convert --foo=bar to --foo bar
parse(opt :: value :: tail)

case value :: tail if value.startsWith("-") =>
SparkSubmit.printErrorAndExit(s"Unrecognized option '$value'.")

case value :: tail =>
primaryResource = if (!SparkSubmit.isShell(value) && !SparkSubmit.isInternal(value)) {
Utils.resolveURI(value).toString
if (inSparkOpts) {
value match {
// convert --foo=bar to --foo bar
case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 =>
val parts = v.split("=")
parse(Seq(parts(0), parts(1)) ++ tail)
case v if v.startsWith("-") =>
val errMessage = s"Unrecognized option '$value'."
SparkSubmit.printErrorAndExit(errMessage)
case v =>
primaryResource =
if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) {
Utils.resolveURI(v).toString
} else {
v
}
inSparkOpts = false
isPython = SparkSubmit.isPython(v)
parse(tail)
}
} else {
value
if (!value.isEmpty) {
childArgs += value
}
parse(tail)
}
isPython = SparkSubmit.isPython(value)
childArgs ++= tail.filter(_.nonEmpty)

case Nil =>
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,10 +101,9 @@ class SparkSubmitSuite extends FunSuite with Matchers {
"--class", "Foo",
"userjar.jar",
"some",
"--class", "Bar",
"--weird", "args")
val appArgs = new SparkSubmitArguments(clArgs)
appArgs.childArgs should be (Seq("some", "--class", "Bar", "--weird", "args"))
appArgs.childArgs should be (Seq("some", "--weird", "args"))
}

test("handles YARN cluster mode") {
Expand Down
2 changes: 1 addition & 1 deletion sbin/start-thriftserver.sh
Original file line number Diff line number Diff line change
Expand Up @@ -33,4 +33,4 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
fi

CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2"
exec "$FWDIR"/bin/spark-submit --class $CLASS $@ spark-internal
exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,14 @@

package org.apache.spark.sql.hive.thriftserver

import scala.collection.JavaConversions._

import org.apache.commons.logging.LogFactory
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService
import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor}

import org.apache.spark.sql.Logging
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
Expand All @@ -45,6 +48,9 @@ private[hive] object HiveThriftServer2 extends Logging {

// Set all properties specified via command line.
val hiveConf: HiveConf = ss.getConf
hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) =>
logger.debug(s"HiveConf var: $k=$v")
}

SessionState.start(ss)

Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,9 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils {

override def beforeAll() {
val pb = new ProcessBuilder(
"../../bin/spark-class",
"-Dspark.master=local",
SparkSQLCLIDriver.getClass.getCanonicalName.stripSuffix("$"),
"../../bin/spark-sql",
"--master",
"local",
"--hiveconf",
s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true",
"--hiveconf",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite}

import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.util.getTempFilePath
import org.apache.spark.sql.hive.test.TestHive

/**
* Test for the HiveThriftServer2 using JDBC.
Expand Down Expand Up @@ -58,16 +57,14 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt
// hard to clean up Hive resources entirely, so we just start a new process and kill
// that process for cleanup.
val defaultArgs = Seq(
"../../bin/spark-class",
"-Dspark.master=local",
HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$"),
"../../sbin/start-thriftserver.sh",
"--master local",
"--hiveconf",
"hive.root.logger=INFO,console",
"--hiveconf",
"\"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH +
";create=true\"",
s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true",
"--hiveconf",
"\"hive.metastore.warehouse.dir=" + WAREHOUSE_PATH + "\"")
s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH")
val pb = new ProcessBuilder(defaultArgs ++ args)
process = pb.start()
inputReader = new BufferedReader(new InputStreamReader(process.getInputStream))
Expand Down Expand Up @@ -121,7 +118,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt

def getConnection: Connection = {
val connectURI = s"jdbc:hive2://localhost:$PORT/"
DriverManager.getConnection(connectURI, "", "")
DriverManager.getConnection(connectURI, System.getProperty("user.name"), "")
}

def createStatement(): Statement = getConnection.createStatement()
Expand Down

0 comments on commit 090beea

Please sign in to comment.