[SPARK-19810][BUILD][CORE] Remove support for Scala 2.10
## What changes were proposed in this pull request? - Remove Scala 2.10 build profiles and support - Replace some 2.10 support in scripts with commented placeholders for 2.12 later - Remove deprecated API calls from 2.10 support - Remove usages of deprecated context bounds where possible - Remove Scala 2.10 workarounds like ScalaReflectionLock - Other minor Scala warning fixes ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #17150 from srowen/SPARK-19810.
This commit is contained in:
parent
e08d06b37b
commit
425c4ada4c
|
@ -113,7 +113,7 @@ sparkR.stop <- function() {
|
|||
#' list(spark.executor.memory="4g"),
|
||||
#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"),
|
||||
#' c("one.jar", "two.jar", "three.jar"),
|
||||
#' c("com.databricks:spark-avro_2.10:2.0.1"))
|
||||
#' c("com.databricks:spark-avro_2.11:2.0.1"))
|
||||
#'}
|
||||
#' @note sparkR.init since 1.4.0
|
||||
sparkR.init <- function(
|
||||
|
@ -357,7 +357,7 @@ sparkRHive.init <- function(jsc = NULL) {
|
|||
#' sparkR.session("yarn-client", "SparkR", "/home/spark",
|
||||
#' list(spark.executor.memory="4g"),
|
||||
#' c("one.jar", "two.jar", "three.jar"),
|
||||
#' c("com.databricks:spark-avro_2.10:2.0.1"))
|
||||
#' c("com.databricks:spark-avro_2.11:2.0.1"))
|
||||
#' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g")
|
||||
#'}
|
||||
#' @note sparkR.session since 2.0.0
|
||||
|
|
|
@ -37,7 +37,7 @@ test_that("multiple packages don't produce a warning", {
|
|||
|
||||
test_that("sparkJars sparkPackages as character vectors", {
|
||||
args <- generateSparkSubmitArgs("", "", c("one.jar", "two.jar", "three.jar"), "",
|
||||
c("com.databricks:spark-avro_2.10:2.0.1"))
|
||||
c("com.databricks:spark-avro_2.11:2.0.1"))
|
||||
expect_match(args, "--jars one.jar,two.jar,three.jar")
|
||||
expect_match(args, "--packages com.databricks:spark-avro_2.10:2.0.1")
|
||||
expect_match(args, "--packages com.databricks:spark-avro_2.11:2.0.1")
|
||||
})
|
||||
|
|
|
@ -35,21 +35,21 @@ if [%SPARK_ENV_LOADED%] == [] (
|
|||
|
||||
rem Setting SPARK_SCALA_VERSION if not already set.
|
||||
|
||||
set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11"
|
||||
set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.10"
|
||||
rem set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11"
|
||||
rem set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.12"
|
||||
|
||||
if [%SPARK_SCALA_VERSION%] == [] (
|
||||
|
||||
if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% (
|
||||
echo "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected."
|
||||
echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd."
|
||||
exit 1
|
||||
)
|
||||
if exist %ASSEMBLY_DIR2% (
|
||||
rem if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% (
|
||||
rem echo "Presence of build for multiple Scala versions detected."
|
||||
rem echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd."
|
||||
rem exit 1
|
||||
rem )
|
||||
rem if exist %ASSEMBLY_DIR2% (
|
||||
set SPARK_SCALA_VERSION=2.11
|
||||
) else (
|
||||
set SPARK_SCALA_VERSION=2.10
|
||||
)
|
||||
rem ) else (
|
||||
rem set SPARK_SCALA_VERSION=2.12
|
||||
rem )
|
||||
)
|
||||
exit /b 0
|
||||
|
||||
|
|
|
@ -46,18 +46,18 @@ fi
|
|||
|
||||
if [ -z "$SPARK_SCALA_VERSION" ]; then
|
||||
|
||||
ASSEMBLY_DIR2="${SPARK_HOME}/assembly/target/scala-2.11"
|
||||
ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.10"
|
||||
#ASSEMBLY_DIR2="${SPARK_HOME}/assembly/target/scala-2.11"
|
||||
#ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.12"
|
||||
|
||||
if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then
|
||||
echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2
|
||||
echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2
|
||||
exit 1
|
||||
fi
|
||||
#if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then
|
||||
# echo -e "Presence of build for multiple Scala versions detected." 1>&2
|
||||
# echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2
|
||||
# exit 1
|
||||
#fi
|
||||
|
||||
if [ -d "$ASSEMBLY_DIR2" ]; then
|
||||
#if [ -d "$ASSEMBLY_DIR2" ]; then
|
||||
export SPARK_SCALA_VERSION="2.11"
|
||||
else
|
||||
export SPARK_SCALA_VERSION="2.10"
|
||||
fi
|
||||
#else
|
||||
# export SPARK_SCALA_VERSION="2.12"
|
||||
#fi
|
||||
fi
|
||||
|
|
|
@ -91,13 +91,13 @@ install_mvn() {
|
|||
|
||||
# Install zinc under the build/ folder
|
||||
install_zinc() {
|
||||
local zinc_path="zinc-0.3.11/bin/zinc"
|
||||
local zinc_path="zinc-0.3.15/bin/zinc"
|
||||
[ ! -f "${_DIR}/${zinc_path}" ] && ZINC_INSTALL_FLAG=1
|
||||
local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.typesafe.com}
|
||||
|
||||
install_app \
|
||||
"${TYPESAFE_MIRROR}/zinc/0.3.11" \
|
||||
"zinc-0.3.11.tgz" \
|
||||
"${TYPESAFE_MIRROR}/zinc/0.3.15" \
|
||||
"zinc-0.3.15.tgz" \
|
||||
"${zinc_path}"
|
||||
ZINC_BIN="${_DIR}/${zinc_path}"
|
||||
}
|
||||
|
|
|
@ -201,7 +201,8 @@ trait AccumulableParam[R, T] extends Serializable {
|
|||
|
||||
@deprecated("use AccumulatorV2", "2.0.0")
|
||||
private[spark] class
|
||||
GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T]
|
||||
GrowableAccumulableParam[R : ClassTag, T]
|
||||
(implicit rg: R => Growable[T] with TraversableOnce[T] with Serializable)
|
||||
extends AccumulableParam[R, T] {
|
||||
|
||||
def addAccumulator(growable: R, elem: T): R = {
|
||||
|
|
|
@ -183,8 +183,6 @@ class SparkContext(config: SparkConf) extends Logging {
|
|||
// log out Spark Version in Spark driver log
|
||||
logInfo(s"Running Spark version $SPARK_VERSION")
|
||||
|
||||
warnDeprecatedVersions()
|
||||
|
||||
/* ------------------------------------------------------------------------------------- *
|
||||
| Private variables. These variables keep the internal state of the context, and are |
|
||||
| not accessible by the outside world. They're mutable since we want to initialize all |
|
||||
|
@ -349,13 +347,6 @@ class SparkContext(config: SparkConf) extends Logging {
|
|||
value
|
||||
}
|
||||
|
||||
private def warnDeprecatedVersions(): Unit = {
|
||||
val javaVersion = System.getProperty("java.version").split("[+.\\-]+", 3)
|
||||
if (scala.util.Properties.releaseVersion.exists(_.startsWith("2.10"))) {
|
||||
logWarning("Support for Scala 2.10 is deprecated as of Spark 2.1.0")
|
||||
}
|
||||
}
|
||||
|
||||
/** Control our logLevel. This overrides any user-defined log settings.
|
||||
* @param logLevel The desired log level as a string.
|
||||
* Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN
|
||||
|
@ -1396,6 +1387,8 @@ class SparkContext(config: SparkConf) extends Logging {
|
|||
@deprecated("use AccumulatorV2", "2.0.0")
|
||||
def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T]
|
||||
(initialValue: R): Accumulable[R, T] = {
|
||||
// TODO the context bound (<%) above should be replaced with simple type bound and implicit
|
||||
// conversion but is a breaking change. This should be fixed in Spark 3.x.
|
||||
val param = new GrowableAccumulableParam[R, T]
|
||||
val acc = new Accumulable(initialValue, param)
|
||||
cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc))
|
||||
|
@ -2605,9 +2598,9 @@ object SparkContext extends Logging {
|
|||
*/
|
||||
private[spark] val LEGACY_DRIVER_IDENTIFIER = "<driver>"
|
||||
|
||||
private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T])
|
||||
private implicit def arrayToArrayWritable[T <: Writable : ClassTag](arr: Traversable[T])
|
||||
: ArrayWritable = {
|
||||
def anyToWritable[U <% Writable](u: U): Writable = u
|
||||
def anyToWritable[U <: Writable](u: U): Writable = u
|
||||
|
||||
new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]],
|
||||
arr.map(x => anyToWritable(x)).toArray)
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
package org.apache.spark.rdd
|
||||
|
||||
import scala.reflect.{classTag, ClassTag}
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
import org.apache.hadoop.io.Writable
|
||||
import org.apache.hadoop.io.compress.CompressionCodec
|
||||
|
@ -39,40 +39,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
|
|||
extends Logging
|
||||
with Serializable {
|
||||
|
||||
private val keyWritableClass =
|
||||
if (_keyWritableClass == null) {
|
||||
// pre 1.3.0, we need to use Reflection to get the Writable class
|
||||
getWritableClass[K]()
|
||||
} else {
|
||||
_keyWritableClass
|
||||
}
|
||||
|
||||
private val valueWritableClass =
|
||||
if (_valueWritableClass == null) {
|
||||
// pre 1.3.0, we need to use Reflection to get the Writable class
|
||||
getWritableClass[V]()
|
||||
} else {
|
||||
_valueWritableClass
|
||||
}
|
||||
|
||||
private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = {
|
||||
val c = {
|
||||
if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) {
|
||||
classTag[T].runtimeClass
|
||||
} else {
|
||||
// We get the type of the Writable class by looking at the apply method which converts
|
||||
// from T to Writable. Since we have two apply methods we filter out the one which
|
||||
// is not of the form "java.lang.Object apply(java.lang.Object)"
|
||||
implicitly[T => Writable].getClass.getDeclaredMethods().filter(
|
||||
m => m.getReturnType().toString != "class java.lang.Object" &&
|
||||
m.getName() == "apply")(0).getReturnType
|
||||
|
||||
}
|
||||
// TODO: use something like WritableConverter to avoid reflection
|
||||
}
|
||||
c.asInstanceOf[Class[_ <: Writable]]
|
||||
}
|
||||
|
||||
// TODO the context bound (<%) above should be replaced with simple type bound and implicit
|
||||
// conversion but is a breaking change. This should be fixed in Spark 3.x.
|
||||
|
||||
/**
|
||||
* Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key
|
||||
|
@ -90,24 +58,24 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
|
|||
// valueWritableClass at the compile time. To implement that, we need to add type parameters to
|
||||
// SequenceFileRDDFunctions. however, SequenceFileRDDFunctions is a public class so it will be a
|
||||
// breaking change.
|
||||
val convertKey = self.keyClass != keyWritableClass
|
||||
val convertValue = self.valueClass != valueWritableClass
|
||||
val convertKey = self.keyClass != _keyWritableClass
|
||||
val convertValue = self.valueClass != _valueWritableClass
|
||||
|
||||
logInfo("Saving as sequence file of type (" + keyWritableClass.getSimpleName + "," +
|
||||
valueWritableClass.getSimpleName + ")" )
|
||||
logInfo("Saving as sequence file of type " +
|
||||
s"(${_keyWritableClass.getSimpleName},${_valueWritableClass.getSimpleName})" )
|
||||
val format = classOf[SequenceFileOutputFormat[Writable, Writable]]
|
||||
val jobConf = new JobConf(self.context.hadoopConfiguration)
|
||||
if (!convertKey && !convertValue) {
|
||||
self.saveAsHadoopFile(path, keyWritableClass, valueWritableClass, format, jobConf, codec)
|
||||
self.saveAsHadoopFile(path, _keyWritableClass, _valueWritableClass, format, jobConf, codec)
|
||||
} else if (!convertKey && convertValue) {
|
||||
self.map(x => (x._1, anyToWritable(x._2))).saveAsHadoopFile(
|
||||
path, keyWritableClass, valueWritableClass, format, jobConf, codec)
|
||||
path, _keyWritableClass, _valueWritableClass, format, jobConf, codec)
|
||||
} else if (convertKey && !convertValue) {
|
||||
self.map(x => (anyToWritable(x._1), x._2)).saveAsHadoopFile(
|
||||
path, keyWritableClass, valueWritableClass, format, jobConf, codec)
|
||||
path, _keyWritableClass, _valueWritableClass, format, jobConf, codec)
|
||||
} else if (convertKey && convertValue) {
|
||||
self.map(x => (anyToWritable(x._1), anyToWritable(x._2))).saveAsHadoopFile(
|
||||
path, keyWritableClass, valueWritableClass, format, jobConf, codec)
|
||||
path, _keyWritableClass, _valueWritableClass, format, jobConf, codec)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -125,9 +125,9 @@ private[spark] object RpcTimeout {
|
|||
var foundProp: Option[(String, String)] = None
|
||||
while (itr.hasNext && foundProp.isEmpty) {
|
||||
val propKey = itr.next()
|
||||
conf.getOption(propKey).foreach { prop => foundProp = Some(propKey, prop) }
|
||||
conf.getOption(propKey).foreach { prop => foundProp = Some((propKey, prop)) }
|
||||
}
|
||||
val finalProp = foundProp.getOrElse(timeoutPropList.head, defaultValue)
|
||||
val finalProp = foundProp.getOrElse((timeoutPropList.head, defaultValue))
|
||||
val timeout = { Utils.timeStringAsSeconds(finalProp._2).seconds }
|
||||
new RpcTimeout(timeout, finalProp._1)
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ private[spark] object JettyUtils extends Logging {
|
|||
// implicit conversion from many types of functions to jetty Handlers.
|
||||
type Responder[T] = HttpServletRequest => T
|
||||
|
||||
class ServletParams[T <% AnyRef](val responder: Responder[T],
|
||||
class ServletParams[T <: AnyRef](val responder: Responder[T],
|
||||
val contentType: String,
|
||||
val extractFn: T => String = (in: Any) => in.toString) {}
|
||||
|
||||
|
@ -68,7 +68,7 @@ private[spark] object JettyUtils extends Logging {
|
|||
implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] =
|
||||
new ServletParams(responder, "text/plain")
|
||||
|
||||
def createServlet[T <% AnyRef](
|
||||
def createServlet[T <: AnyRef](
|
||||
servletParams: ServletParams[T],
|
||||
securityMgr: SecurityManager,
|
||||
conf: SparkConf): HttpServlet = {
|
||||
|
@ -113,7 +113,7 @@ private[spark] object JettyUtils extends Logging {
|
|||
}
|
||||
|
||||
/** Create a context handler that responds to a request with the given path prefix */
|
||||
def createServletHandler[T <% AnyRef](
|
||||
def createServletHandler[T <: AnyRef](
|
||||
path: String,
|
||||
servletParams: ServletParams[T],
|
||||
securityMgr: SecurityManager,
|
||||
|
|
|
@ -125,16 +125,16 @@ private[spark] object FileAppender extends Logging {
|
|||
val validatedParams: Option[(Long, String)] = rollingInterval match {
|
||||
case "daily" =>
|
||||
logInfo(s"Rolling executor logs enabled for $file with daily rolling")
|
||||
Some(24 * 60 * 60 * 1000L, "--yyyy-MM-dd")
|
||||
Some((24 * 60 * 60 * 1000L, "--yyyy-MM-dd"))
|
||||
case "hourly" =>
|
||||
logInfo(s"Rolling executor logs enabled for $file with hourly rolling")
|
||||
Some(60 * 60 * 1000L, "--yyyy-MM-dd--HH")
|
||||
Some((60 * 60 * 1000L, "--yyyy-MM-dd--HH"))
|
||||
case "minutely" =>
|
||||
logInfo(s"Rolling executor logs enabled for $file with rolling every minute")
|
||||
Some(60 * 1000L, "--yyyy-MM-dd--HH-mm")
|
||||
Some((60 * 1000L, "--yyyy-MM-dd--HH-mm"))
|
||||
case IntParam(seconds) =>
|
||||
logInfo(s"Rolling executor logs enabled for $file with rolling $seconds seconds")
|
||||
Some(seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss")
|
||||
Some((seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss"))
|
||||
case _ =>
|
||||
logWarning(s"Illegal interval for rolling executor logs [$rollingInterval], " +
|
||||
s"rolling logs not enabled")
|
||||
|
|
|
@ -113,11 +113,11 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
|
|||
|
||||
val normalFile = new File(normalDir, "part-00000")
|
||||
val normalContent = sc.sequenceFile[String, String](normalDir).collect
|
||||
assert(normalContent === Array.fill(100)("abc", "abc"))
|
||||
assert(normalContent === Array.fill(100)(("abc", "abc")))
|
||||
|
||||
val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension)
|
||||
val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect
|
||||
assert(compressedContent === Array.fill(100)("abc", "abc"))
|
||||
assert(compressedContent === Array.fill(100)(("abc", "abc")))
|
||||
|
||||
assert(compressedFile.length < normalFile.length)
|
||||
}
|
||||
|
|
|
@ -93,8 +93,8 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
|
|||
|
||||
test("add dependencies works correctly") {
|
||||
val md = SparkSubmitUtils.getModuleDescriptor
|
||||
val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.10:0.1," +
|
||||
"com.databricks:spark-avro_2.10:0.1")
|
||||
val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.11:0.1," +
|
||||
"com.databricks:spark-avro_2.11:0.1")
|
||||
|
||||
SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default")
|
||||
assert(md.getDependencies.length === 2)
|
||||
|
@ -196,7 +196,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
|
|||
SparkSubmitUtils.buildIvySettings(None, None),
|
||||
isTest = true)
|
||||
assert(path === "", "should return empty path")
|
||||
val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.10", "1.2.0")
|
||||
val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.11", "1.2.0")
|
||||
IvyTestUtils.withRepository(main, None, None) { repo =>
|
||||
val files = SparkSubmitUtils.resolveMavenCoordinates(
|
||||
coordinates + "," + main.toString,
|
||||
|
|
|
@ -80,6 +80,7 @@ class MockWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) extend
|
|||
case Some(appId) =>
|
||||
apps.remove(appId)
|
||||
master.send(UnregisterApplication(appId))
|
||||
case None =>
|
||||
}
|
||||
driverIdToAppId.remove(driverId)
|
||||
}
|
||||
|
@ -575,7 +576,7 @@ class MasterSuite extends SparkFunSuite
|
|||
override val rpcEnv: RpcEnv = master.rpcEnv
|
||||
|
||||
override def receive: PartialFunction[Any, Unit] = {
|
||||
case KillExecutor(_, appId, execId) => killedExecutors.add(appId, execId)
|
||||
case KillExecutor(_, appId, execId) => killedExecutors.add((appId, execId))
|
||||
case KillDriver(driverId) => killedDrivers.add(driverId)
|
||||
}
|
||||
})
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.{CountDownLatch, TimeUnit}
|
|||
|
||||
import scala.collection.mutable.Map
|
||||
import scala.concurrent.duration._
|
||||
import scala.language.postfixOps
|
||||
|
||||
import org.mockito.ArgumentCaptor
|
||||
import org.mockito.Matchers.{any, eq => meq}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.spark.rdd
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.language.postfixOps
|
||||
|
||||
import org.scalatest.concurrent.Eventually.{eventually, interval, timeout}
|
||||
|
||||
|
|
|
@ -115,7 +115,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
|
|||
locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = {
|
||||
if (!hasDequeuedSpeculatedTask) {
|
||||
hasDequeuedSpeculatedTask = true
|
||||
Some(0, TaskLocality.PROCESS_LOCAL)
|
||||
Some((0, TaskLocality.PROCESS_LOCAL))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
|
|
@ -299,7 +299,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
|
|||
val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1")
|
||||
val d3 = d.map { i => w(i) -> (0 to (i % 5)) }.setName("shuffle input 2")
|
||||
val d4 = d2.cogroup(d3, numSlices).map { case (k, (v1, v2)) =>
|
||||
w(k) -> (v1.size, v2.size)
|
||||
(w(k), (v1.size, v2.size))
|
||||
}
|
||||
d4.setName("A Cogroup")
|
||||
d4.collectAsMap()
|
||||
|
|
|
@ -86,8 +86,8 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
|
|||
Seq("exec1", "exec2").foreach { exec =>
|
||||
assert(
|
||||
execToFailures(exec).taskToFailureCountAndFailureTime === Map(
|
||||
0 -> (1, 0),
|
||||
1 -> (1, 0)
|
||||
0 -> ((1, 0)),
|
||||
1 -> ((1, 0))
|
||||
)
|
||||
)
|
||||
}
|
||||
|
|
|
@ -276,7 +276,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
|
|||
}
|
||||
|
||||
test("kryo with collect for specialized tuples") {
|
||||
assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11))
|
||||
assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === ((1, 11)))
|
||||
}
|
||||
|
||||
test("kryo with SerializableHyperLogLog") {
|
||||
|
@ -475,7 +475,7 @@ class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSpar
|
|||
val deserializationStream = serInstance.deserializeStream(new ByteArrayInputStream(worldWorld))
|
||||
assert(deserializationStream.readValue[Any]() === world)
|
||||
deserializationStream.close()
|
||||
assert(serInstance.deserialize[Any](helloHello) === (hello, hello))
|
||||
assert(serInstance.deserialize[Any](helloHello) === ((hello, hello)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -115,7 +115,7 @@ class TimeStampedHashMapSuite extends SparkFunSuite {
|
|||
testMap2("k1") = "v1"
|
||||
testMap2 --= keys
|
||||
assert(testMap2.size === 1)
|
||||
assert(testMap2.iterator.toSeq.head === ("k1", "v1"))
|
||||
assert(testMap2.iterator.toSeq.head === (("k1", "v1")))
|
||||
|
||||
// +
|
||||
val testMap3 = testMap2 + (("k0", "v0"))
|
||||
|
|
|
@ -58,10 +58,10 @@ class VersionUtilsSuite extends SparkFunSuite {
|
|||
}
|
||||
|
||||
test("Parse Spark major and minor versions") {
|
||||
assert(majorMinorVersion("2.0") === (2, 0))
|
||||
assert(majorMinorVersion("12.10.11") === (12, 10))
|
||||
assert(majorMinorVersion("2.0.1-SNAPSHOT") === (2, 0))
|
||||
assert(majorMinorVersion("2.0.x") === (2, 0))
|
||||
assert(majorMinorVersion("2.0") === ((2, 0)))
|
||||
assert(majorMinorVersion("12.10.11") === ((12, 10)))
|
||||
assert(majorMinorVersion("2.0.1-SNAPSHOT") === ((2, 0)))
|
||||
assert(majorMinorVersion("2.0.x") === ((2, 0)))
|
||||
withClue("majorMinorVersion parsing should fail for invalid major version number") {
|
||||
intercept[IllegalArgumentException] {
|
||||
majorMinorVersion("2z.0")
|
||||
|
|
|
@ -181,9 +181,9 @@ class AppendOnlyMapSuite extends SparkFunSuite {
|
|||
// Should be sorted by key
|
||||
assert(it.hasNext)
|
||||
var previous = it.next()
|
||||
assert(previous == (null, "happy new year!"))
|
||||
assert(previous == ((null, "happy new year!")))
|
||||
previous = it.next()
|
||||
assert(previous == ("1", "2014"))
|
||||
assert(previous == (("1", "2014")))
|
||||
while (it.hasNext) {
|
||||
val kv = it.next()
|
||||
assert(kv._1.toInt > previous._1.toInt)
|
||||
|
|
|
@ -388,13 +388,13 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext {
|
|||
sorter.insertAll(elements)
|
||||
assert(sorter.numSpills > 0, "sorter did not spill")
|
||||
val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList))
|
||||
assert(iter.next() === (0, Nil))
|
||||
assert(iter.next() === (1, List((1, 1))))
|
||||
assert(iter.next() === (2, (0 until 1000).map(x => (2, 2)).toList))
|
||||
assert(iter.next() === (3, Nil))
|
||||
assert(iter.next() === (4, Nil))
|
||||
assert(iter.next() === (5, List((5, 5))))
|
||||
assert(iter.next() === (6, Nil))
|
||||
assert(iter.next() === ((0, Nil)))
|
||||
assert(iter.next() === ((1, List((1, 1)))))
|
||||
assert(iter.next() === ((2, (0 until 1000).map(x => (2, 2)).toList)))
|
||||
assert(iter.next() === ((3, Nil)))
|
||||
assert(iter.next() === ((4, Nil)))
|
||||
assert(iter.next() === ((5, List((5, 5)))))
|
||||
assert(iter.next() === ((6, Nil)))
|
||||
sorter.stop()
|
||||
}
|
||||
|
||||
|
|
|
@ -75,7 +75,7 @@ class OpenHashMapSuite extends SparkFunSuite with Matchers {
|
|||
for ((k, v) <- map) {
|
||||
set.add((k, v))
|
||||
}
|
||||
val expected = (1 to 1000).map(x => (x.toString, x)) :+ (null.asInstanceOf[String], -1)
|
||||
val expected = (1 to 1000).map(x => (x.toString, x)) :+ ((null.asInstanceOf[String], -1))
|
||||
assert(set === expected.toSet)
|
||||
}
|
||||
|
||||
|
@ -103,7 +103,8 @@ class OpenHashMapSuite extends SparkFunSuite with Matchers {
|
|||
for ((k, v) <- map) {
|
||||
set.add((k, v))
|
||||
}
|
||||
val expected = (1 to 1000).map(_.toString).map(x => (x, x)) :+ (null.asInstanceOf[String], "-1")
|
||||
val expected =
|
||||
(1 to 1000).map(_.toString).map(x => (x, x)) :+ ((null.asInstanceOf[String], "-1"))
|
||||
assert(set === expected.toSet)
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
set -e
|
||||
|
||||
VALID_VERSIONS=( 2.10 2.11 )
|
||||
VALID_VERSIONS=( 2.11 2.12 )
|
||||
|
||||
usage() {
|
||||
echo "Usage: $(basename $0) [-h|--help] <version>
|
||||
|
@ -45,7 +45,7 @@ check_scala_version() {
|
|||
check_scala_version "$TO_VERSION"
|
||||
|
||||
if [ $TO_VERSION = "2.11" ]; then
|
||||
FROM_VERSION="2.10"
|
||||
FROM_VERSION="2.12"
|
||||
else
|
||||
FROM_VERSION="2.11"
|
||||
fi
|
||||
|
|
|
@ -1,23 +0,0 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# This script exists for backwards compatibility. Use change-scala-version.sh instead.
|
||||
echo "This script is deprecated. Please instead run: change-scala-version.sh 2.10"
|
||||
|
||||
$(dirname $0)/change-scala-version.sh 2.10
|
|
@ -1,23 +0,0 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# This script exists for backwards compatibility. Use change-scala-version.sh instead.
|
||||
echo "This script is deprecated. Please instead run: change-scala-version.sh 2.11"
|
||||
|
||||
$(dirname $0)/change-scala-version.sh 2.11
|
|
@ -155,10 +155,10 @@ if [[ "$1" == "package" ]]; then
|
|||
|
||||
cd spark-$SPARK_VERSION-bin-$NAME
|
||||
|
||||
# TODO There should probably be a flag to make-distribution to allow 2.10 support
|
||||
if [[ $FLAGS == *scala-2.10* ]]; then
|
||||
./dev/change-scala-version.sh 2.10
|
||||
fi
|
||||
# TODO There should probably be a flag to make-distribution to allow 2.12 support
|
||||
#if [[ $FLAGS == *scala-2.12* ]]; then
|
||||
# ./dev/change-scala-version.sh 2.12
|
||||
#fi
|
||||
|
||||
export ZINC_PORT=$ZINC_PORT
|
||||
echo "Creating distribution: $NAME ($FLAGS)"
|
||||
|
@ -305,9 +305,9 @@ if [[ "$1" == "publish-snapshot" ]]; then
|
|||
export ZINC_PORT=$(python -S -c "import random; print random.randrange(3030,4030)")
|
||||
|
||||
$MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES deploy
|
||||
./dev/change-scala-version.sh 2.10
|
||||
$MVN -DzincPort=$ZINC_PORT -Dscala-2.10 --settings $tmp_settings \
|
||||
-DskipTests $PUBLISH_PROFILES clean deploy
|
||||
#./dev/change-scala-version.sh 2.12
|
||||
#$MVN -DzincPort=$ZINC_PORT -Pscala-2.12 --settings $tmp_settings \
|
||||
# -DskipTests $PUBLISH_PROFILES clean deploy
|
||||
|
||||
# Clean-up Zinc nailgun process
|
||||
/usr/sbin/lsof -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill
|
||||
|
@ -342,16 +342,13 @@ if [[ "$1" == "publish-release" ]]; then
|
|||
|
||||
$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $PUBLISH_PROFILES clean install
|
||||
|
||||
./dev/change-scala-version.sh 2.10
|
||||
|
||||
$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -Dscala-2.10 \
|
||||
-DskipTests $PUBLISH_PROFILES clean install
|
||||
#./dev/change-scala-version.sh 2.12
|
||||
#$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -Pscala-2.12 \
|
||||
# -DskipTests $PUBLISH_PROFILES clean install
|
||||
|
||||
# Clean-up Zinc nailgun process
|
||||
/usr/sbin/lsof -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill
|
||||
|
||||
./dev/change-version-to-2.10.sh
|
||||
|
||||
pushd $tmp_repo/org/apache/spark
|
||||
|
||||
# Remove any extra files generated during install
|
||||
|
|
|
@ -91,14 +91,6 @@ like ZooKeeper and Hadoop itself.
|
|||
|
||||
./build/mvn -Pmesos -DskipTests clean package
|
||||
|
||||
## Building for Scala 2.10
|
||||
To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property:
|
||||
|
||||
./dev/change-scala-version.sh 2.10
|
||||
./build/mvn -Pyarn -Dscala-2.10 -DskipTests clean package
|
||||
|
||||
Note that support for Scala 2.10 is deprecated as of Spark 2.1.0 and may be removed in Spark 2.3.0.
|
||||
|
||||
## Building submodules individually
|
||||
|
||||
It's possible to build Spark sub-modules using the `mvn -pl` option.
|
||||
|
|
|
@ -31,8 +31,7 @@ uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scal
|
|||
({{site.SCALA_BINARY_VERSION}}.x).
|
||||
|
||||
Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0.
|
||||
|
||||
Note that support for Scala 2.10 is deprecated as of Spark 2.1.0, and may be removed in Spark 2.3.0.
|
||||
Support for Scala 2.10 was removed as of 2.3.0.
|
||||
|
||||
# Running the Examples and Shell
|
||||
|
||||
|
|
|
@ -52,7 +52,7 @@ object AggregateMessagesExample {
|
|||
triplet => { // Map Function
|
||||
if (triplet.srcAttr > triplet.dstAttr) {
|
||||
// Send message to destination vertex containing counter and age
|
||||
triplet.sendToDst(1, triplet.srcAttr)
|
||||
triplet.sendToDst((1, triplet.srcAttr))
|
||||
}
|
||||
},
|
||||
// Add counter and age
|
||||
|
|
|
@ -38,7 +38,7 @@ abstract class AbstractParams[T: TypeTag] {
|
|||
*/
|
||||
override def toString: String = {
|
||||
val tpe = tag.tpe
|
||||
val allAccessors = tpe.declarations.collect {
|
||||
val allAccessors = tpe.decls.collect {
|
||||
case m: MethodSymbol if m.isCaseAccessor => m
|
||||
}
|
||||
val mirror = runtimeMirror(getClass.getClassLoader)
|
||||
|
|
|
@ -29,8 +29,6 @@ import org.apache.spark.sql.types._
|
|||
object SparkSQLExample {
|
||||
|
||||
// $example on:create_ds$
|
||||
// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit,
|
||||
// you can use custom classes that implement the Product interface
|
||||
case class Person(name: String, age: Long)
|
||||
// $example off:create_ds$
|
||||
|
||||
|
|
|
@ -259,8 +259,8 @@ object KafkaRDD {
|
|||
messageHandler: MessageAndMetadata[K, V] => R
|
||||
): KafkaRDD[K, V, U, T, R] = {
|
||||
val leaders = untilOffsets.map { case (tp, lo) =>
|
||||
tp -> (lo.host, lo.port)
|
||||
}.toMap
|
||||
tp -> ((lo.host, lo.port))
|
||||
}
|
||||
|
||||
val offsetRanges = fromOffsets.map { case (tp, fo) =>
|
||||
val uo = untilOffsets(tp)
|
||||
|
|
|
@ -63,5 +63,5 @@ object EdgeContext {
|
|||
* }}}
|
||||
*/
|
||||
def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]): Some[(VertexId, VertexId, VD, VD, ED)] =
|
||||
Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr)
|
||||
Some((edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr))
|
||||
}
|
||||
|
|
|
@ -229,17 +229,17 @@ abstract class AbstractCommandBuilder {
|
|||
return scala;
|
||||
}
|
||||
String sparkHome = getSparkHome();
|
||||
File scala210 = new File(sparkHome, "launcher/target/scala-2.10");
|
||||
//File scala212 = new File(sparkHome, "launcher/target/scala-2.12");
|
||||
File scala211 = new File(sparkHome, "launcher/target/scala-2.11");
|
||||
checkState(!scala210.isDirectory() || !scala211.isDirectory(),
|
||||
"Presence of build for both scala versions (2.10 and 2.11) detected.\n" +
|
||||
"Either clean one of them or set SPARK_SCALA_VERSION in your environment.");
|
||||
if (scala210.isDirectory()) {
|
||||
return "2.10";
|
||||
} else {
|
||||
checkState(scala211.isDirectory(), "Cannot find any build directories.");
|
||||
return "2.11";
|
||||
}
|
||||
//checkState(!scala210.isDirectory() || !scala211.isDirectory(),
|
||||
// "Presence of build for multiple Scala versions detected.\n" +
|
||||
// "Either clean one of them or set SPARK_SCALA_VERSION in your environment.");
|
||||
//if (scala212.isDirectory()) {
|
||||
// return "2.12";
|
||||
//} else {
|
||||
checkState(scala211.isDirectory(), "Cannot find any build directories.");
|
||||
return "2.11";
|
||||
//}
|
||||
}
|
||||
|
||||
String getSparkHome() {
|
||||
|
|
|
@ -856,7 +856,7 @@ object SparseMatrix {
|
|||
var prevRow = -1
|
||||
var prevVal = 0.0
|
||||
// Append a dummy entry to include the last one at the end of the loop.
|
||||
(sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) =>
|
||||
(sortedEntries.view :+ ((numRows, numCols, 1.0))).foreach { case (i, j, v) =>
|
||||
if (v != 0) {
|
||||
if (i == prevRow && j == prevCol) {
|
||||
prevVal += v
|
||||
|
|
|
@ -633,22 +633,22 @@ class MatricesSuite extends SparkMLFunSuite {
|
|||
dnMap.put((i, j), value)
|
||||
}
|
||||
assert(dnMap.size === 6)
|
||||
assert(dnMap(0, 0) === 1.0)
|
||||
assert(dnMap(1, 0) === 2.0)
|
||||
assert(dnMap(2, 0) === 0.0)
|
||||
assert(dnMap(0, 1) === 0.0)
|
||||
assert(dnMap(1, 1) === 4.0)
|
||||
assert(dnMap(2, 1) === 5.0)
|
||||
assert(dnMap((0, 0)) === 1.0)
|
||||
assert(dnMap((1, 0)) === 2.0)
|
||||
assert(dnMap((2, 0)) === 0.0)
|
||||
assert(dnMap((0, 1)) === 0.0)
|
||||
assert(dnMap((1, 1)) === 4.0)
|
||||
assert(dnMap((2, 1)) === 5.0)
|
||||
|
||||
val spMap = MutableMap[(Int, Int), Double]()
|
||||
sp.foreachActive { (i, j, value) =>
|
||||
spMap.put((i, j), value)
|
||||
}
|
||||
assert(spMap.size === 4)
|
||||
assert(spMap(0, 0) === 1.0)
|
||||
assert(spMap(1, 0) === 2.0)
|
||||
assert(spMap(1, 1) === 4.0)
|
||||
assert(spMap(2, 1) === 5.0)
|
||||
assert(spMap((0, 0)) === 1.0)
|
||||
assert(spMap((1, 0)) === 2.0)
|
||||
assert(spMap((1, 1)) === 4.0)
|
||||
assert(spMap((2, 1)) === 5.0)
|
||||
}
|
||||
|
||||
test("horzcat, vertcat, eye, speye") {
|
||||
|
|
|
@ -436,7 +436,7 @@ private[ml] object EnsembleModelReadWrite {
|
|||
val treesMetadataRDD: RDD[(Int, (Metadata, Double))] = sql.read.parquet(treesMetadataPath)
|
||||
.select("treeID", "metadata", "weights").as[(Int, String, Double)].rdd.map {
|
||||
case (treeID: Int, json: String, weights: Double) =>
|
||||
treeID -> (DefaultParamsReader.parseMetadata(json, treeClassName), weights)
|
||||
treeID -> ((DefaultParamsReader.parseMetadata(json, treeClassName), weights))
|
||||
}
|
||||
|
||||
val treesMetadataWeights = treesMetadataRDD.sortByKey().values.collect()
|
||||
|
|
|
@ -797,7 +797,7 @@ object SparseMatrix {
|
|||
var prevRow = -1
|
||||
var prevVal = 0.0
|
||||
// Append a dummy entry to include the last one at the end of the loop.
|
||||
(sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) =>
|
||||
(sortedEntries.view :+ ((numRows, numCols, 1.0))).foreach { case (i, j, v) =>
|
||||
if (v != 0) {
|
||||
if (i == prevRow && j == prevCol) {
|
||||
prevVal += v
|
||||
|
|
|
@ -275,7 +275,7 @@ class BlockMatrix @Since("1.3.0") (
|
|||
val rows = blocks.flatMap { case ((blockRowIdx, blockColIdx), mat) =>
|
||||
mat.rowIter.zipWithIndex.map {
|
||||
case (vector, rowIdx) =>
|
||||
blockRowIdx * rowsPerBlock + rowIdx -> (blockColIdx, vector.asBreeze)
|
||||
blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector.asBreeze))
|
||||
}
|
||||
}.groupByKey().map { case (rowIdx, vectors) =>
|
||||
val numberNonZeroPerRow = vectors.map(_._2.activeSize).sum.toDouble / cols.toDouble
|
||||
|
@ -286,7 +286,7 @@ class BlockMatrix @Since("1.3.0") (
|
|||
BDV.zeros[Double](cols)
|
||||
}
|
||||
|
||||
vectors.foreach { case (blockColIdx: Int, vec: BV[Double]) =>
|
||||
vectors.foreach { case (blockColIdx: Int, vec: BV[_]) =>
|
||||
val offset = colsPerBlock * blockColIdx
|
||||
wholeVector(offset until Math.min(cols, offset + colsPerBlock)) := vec
|
||||
}
|
||||
|
|
|
@ -898,7 +898,7 @@ class ALSStorageSuite
|
|||
// check final factor RDD default storage levels
|
||||
val defaultFactorRDDs = sc.getPersistentRDDs.collect {
|
||||
case (id, rdd) if rdd.name == "userFactors" || rdd.name == "itemFactors" =>
|
||||
rdd.name -> (id, rdd.getStorageLevel)
|
||||
rdd.name -> ((id, rdd.getStorageLevel))
|
||||
}.toMap
|
||||
defaultFactorRDDs.foreach { case (_, (id, level)) =>
|
||||
assert(level == StorageLevel.MEMORY_AND_DISK)
|
||||
|
|
|
@ -151,7 +151,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext {
|
|||
// Check: topTopicAssignments
|
||||
// Make sure it assigns a topic to each term appearing in each doc.
|
||||
val topTopicAssignments: Map[Long, (Array[Int], Array[Int])] =
|
||||
model.topicAssignments.collect().map(x => x._1 -> (x._2, x._3)).toMap
|
||||
model.topicAssignments.collect().map(x => x._1 -> ((x._2, x._3))).toMap
|
||||
assert(topTopicAssignments.keys.max < tinyCorpus.length)
|
||||
tinyCorpus.foreach { case (docID: Long, doc: Vector) =>
|
||||
if (topTopicAssignments.contains(docID)) {
|
||||
|
|
|
@ -241,22 +241,22 @@ class MatricesSuite extends SparkFunSuite {
|
|||
dnMap.put((i, j), value)
|
||||
}
|
||||
assert(dnMap.size === 6)
|
||||
assert(dnMap(0, 0) === 1.0)
|
||||
assert(dnMap(1, 0) === 2.0)
|
||||
assert(dnMap(2, 0) === 0.0)
|
||||
assert(dnMap(0, 1) === 0.0)
|
||||
assert(dnMap(1, 1) === 4.0)
|
||||
assert(dnMap(2, 1) === 5.0)
|
||||
assert(dnMap((0, 0)) === 1.0)
|
||||
assert(dnMap((1, 0)) === 2.0)
|
||||
assert(dnMap((2, 0)) === 0.0)
|
||||
assert(dnMap((0, 1)) === 0.0)
|
||||
assert(dnMap((1, 1)) === 4.0)
|
||||
assert(dnMap((2, 1)) === 5.0)
|
||||
|
||||
val spMap = MutableMap[(Int, Int), Double]()
|
||||
sp.foreachActive { (i, j, value) =>
|
||||
spMap.put((i, j), value)
|
||||
}
|
||||
assert(spMap.size === 4)
|
||||
assert(spMap(0, 0) === 1.0)
|
||||
assert(spMap(1, 0) === 2.0)
|
||||
assert(spMap(1, 1) === 4.0)
|
||||
assert(spMap(2, 1) === 5.0)
|
||||
assert(spMap((0, 0)) === 1.0)
|
||||
assert(spMap((1, 0)) === 2.0)
|
||||
assert(spMap((1, 1)) === 4.0)
|
||||
assert(spMap((2, 1)) === 5.0)
|
||||
}
|
||||
|
||||
test("horzcat, vertcat, eye, speye") {
|
||||
|
|
84
pom.xml
84
pom.xml
|
@ -498,7 +498,7 @@
|
|||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>jcl-over-slf4j</artifactId>
|
||||
<version>${slf4j.version}</version>
|
||||
<!-- <scope>runtime</scope> --> <!-- more correct, but scalac 2.10.3 doesn't like it -->
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>log4j</groupId>
|
||||
|
@ -1859,9 +1859,9 @@
|
|||
<version>${antlr4.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>${jline.groupid}</groupId>
|
||||
<groupId>jline</groupId>
|
||||
<artifactId>jline</artifactId>
|
||||
<version>${jline.version}</version>
|
||||
<version>2.12.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
|
@ -1933,6 +1933,7 @@
|
|||
-->
|
||||
<exclude>org.jboss.netty</exclude>
|
||||
<exclude>org.codehaus.groovy</exclude>
|
||||
<exclude>*:*_2.10</exclude>
|
||||
</excludes>
|
||||
<searchTransitive>true</searchTransitive>
|
||||
</bannedDependencies>
|
||||
|
@ -1987,6 +1988,8 @@
|
|||
<arg>-unchecked</arg>
|
||||
<arg>-deprecation</arg>
|
||||
<arg>-feature</arg>
|
||||
<arg>-explaintypes</arg>
|
||||
<arg>-Yno-adapted-args</arg>
|
||||
</args>
|
||||
<jvmArgs>
|
||||
<jvmArg>-Xms1024m</jvmArg>
|
||||
|
@ -2586,15 +2589,27 @@
|
|||
</profile>
|
||||
|
||||
<profile>
|
||||
<id>scala-2.10</id>
|
||||
<id>test-java-home</id>
|
||||
<activation>
|
||||
<property><name>scala-2.10</name></property>
|
||||
<property><name>env.JAVA_HOME</name></property>
|
||||
</activation>
|
||||
<properties>
|
||||
<scala.version>2.10.6</scala.version>
|
||||
<scala.binary.version>2.10</scala.binary.version>
|
||||
<jline.version>${scala.version}</jline.version>
|
||||
<jline.groupid>org.scala-lang</jline.groupid>
|
||||
<test.java.home>${env.JAVA_HOME}</test.java.home>
|
||||
</properties>
|
||||
</profile>
|
||||
|
||||
<!-- Exists for backwards compatibility; profile doesn't do anything -->
|
||||
<profile>
|
||||
<id>scala-2.11</id>
|
||||
</profile>
|
||||
|
||||
<!-- Draft of Scala 2.12 profile for later -->
|
||||
<!--
|
||||
<profile>
|
||||
<id>scala-2.12</id>
|
||||
<properties>
|
||||
<scala.version>2.12.1</scala.version>
|
||||
<scala.binary.version>2.12</scala.binary.version>
|
||||
</properties>
|
||||
<build>
|
||||
<plugins>
|
||||
|
@ -2622,58 +2637,11 @@
|
|||
</plugins>
|
||||
</build>
|
||||
</profile>
|
||||
|
||||
<profile>
|
||||
<id>test-java-home</id>
|
||||
<activation>
|
||||
<property><name>env.JAVA_HOME</name></property>
|
||||
</activation>
|
||||
<properties>
|
||||
<test.java.home>${env.JAVA_HOME}</test.java.home>
|
||||
</properties>
|
||||
</profile>
|
||||
|
||||
<profile>
|
||||
<id>scala-2.11</id>
|
||||
<activation>
|
||||
<property><name>!scala-2.10</name></property>
|
||||
</activation>
|
||||
<properties>
|
||||
<scala.version>2.11.8</scala.version>
|
||||
<scala.binary.version>2.11</scala.binary.version>
|
||||
<jline.version>2.12.1</jline.version>
|
||||
<jline.groupid>jline</jline.groupid>
|
||||
</properties>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-enforcer-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>enforce-versions</id>
|
||||
<goals>
|
||||
<goal>enforce</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<rules>
|
||||
<bannedDependencies>
|
||||
<excludes combine.children="append">
|
||||
<exclude>*:*_2.10</exclude>
|
||||
</excludes>
|
||||
</bannedDependencies>
|
||||
</rules>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</profile>
|
||||
-->
|
||||
|
||||
<!--
|
||||
This is a profile to enable the use of the ASF snapshot and staging repositories
|
||||
during a build. It is useful when testing againt nightly or RC releases of dependencies.
|
||||
during a build. It is useful when testing against nightly or RC releases of dependencies.
|
||||
It MUST NOT be used when building copies of Spark to use in production of for distribution,
|
||||
-->
|
||||
<profile>
|
||||
|
|
|
@ -87,19 +87,11 @@ object SparkBuild extends PomBuild {
|
|||
val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty
|
||||
|
||||
override val profiles = {
|
||||
val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match {
|
||||
Properties.envOrNone("SBT_MAVEN_PROFILES") match {
|
||||
case None => Seq("sbt")
|
||||
case Some(v) =>
|
||||
v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq
|
||||
}
|
||||
|
||||
if (System.getProperty("scala-2.10") == "") {
|
||||
// To activate scala-2.10 profile, replace empty property value to non-empty value
|
||||
// in the same way as Maven which handles -Dname as -Dname=true before executes build process.
|
||||
// see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082
|
||||
System.setProperty("scala-2.10", "true")
|
||||
}
|
||||
profiles
|
||||
}
|
||||
|
||||
Properties.envOrNone("SBT_MAVEN_PROPERTIES") match {
|
||||
|
@ -234,9 +226,7 @@ object SparkBuild extends PomBuild {
|
|||
},
|
||||
|
||||
javacJVMVersion := "1.8",
|
||||
// SBT Scala 2.10 build still doesn't support Java 8, because scalac 2.10 doesn't, but,
|
||||
// it also doesn't touch Java 8 code and it's OK to emit Java 7 bytecode in this case
|
||||
scalacJVMVersion := (if (System.getProperty("scala-2.10") == "true") "1.7" else "1.8"),
|
||||
scalacJVMVersion := "1.8",
|
||||
|
||||
javacOptions in Compile ++= Seq(
|
||||
"-encoding", "UTF-8",
|
||||
|
@ -477,7 +467,6 @@ object OldDeps {
|
|||
|
||||
def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq(
|
||||
name := "old-deps",
|
||||
scalaVersion := "2.10.5",
|
||||
libraryDependencies := allPreviousArtifactKeys.value.flatten
|
||||
)
|
||||
}
|
||||
|
@ -756,13 +745,7 @@ object CopyDependencies {
|
|||
object TestSettings {
|
||||
import BuildCommons._
|
||||
|
||||
private val scalaBinaryVersion =
|
||||
if (System.getProperty("scala-2.10") == "true") {
|
||||
"2.10"
|
||||
} else {
|
||||
"2.11"
|
||||
}
|
||||
|
||||
private val scalaBinaryVersion = "2.11"
|
||||
lazy val settings = Seq (
|
||||
// Fork new JVMs for tests and set Java options for those
|
||||
fork := true,
|
||||
|
|
|
@ -54,7 +54,8 @@ FAILURE_REPORTING_LOCK = Lock()
|
|||
LOGGER = logging.getLogger()
|
||||
|
||||
# Find out where the assembly jars are located.
|
||||
for scala in ["2.11", "2.10"]:
|
||||
# Later, add back 2.12 to this list:
|
||||
for scala in ["2.11"]:
|
||||
build_dir = os.path.join(SPARK_HOME, "assembly", "target", "scala-" + scala)
|
||||
if os.path.isdir(build_dir):
|
||||
SPARK_DIST_CLASSPATH = os.path.join(build_dir, "jars", "*")
|
||||
|
|
26
repl/pom.xml
26
repl/pom.xml
|
@ -32,8 +32,8 @@
|
|||
|
||||
<properties>
|
||||
<sbt.project.name>repl</sbt.project.name>
|
||||
<extra.source.dir>scala-2.10/src/main/scala</extra.source.dir>
|
||||
<extra.testsource.dir>scala-2.10/src/test/scala</extra.testsource.dir>
|
||||
<extra.source.dir>scala-2.11/src/main/scala</extra.source.dir>
|
||||
<extra.testsource.dir>scala-2.11/src/test/scala</extra.testsource.dir>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
|
@ -71,7 +71,7 @@
|
|||
<version>${scala.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>${jline.groupid}</groupId>
|
||||
<groupId>jline</groupId>
|
||||
<artifactId>jline</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
|
@ -170,23 +170,17 @@
|
|||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<!--
|
||||
<profiles>
|
||||
<profile>
|
||||
<id>scala-2.10</id>
|
||||
<activation>
|
||||
<property><name>scala-2.10</name></property>
|
||||
</activation>
|
||||
</profile>
|
||||
|
||||
<profile>
|
||||
<id>scala-2.11</id>
|
||||
<activation>
|
||||
<property><name>!scala-2.10</name></property>
|
||||
</activation>
|
||||
<id>scala-2.12</id>
|
||||
<properties>
|
||||
<extra.source.dir>scala-2.11/src/main/scala</extra.source.dir>
|
||||
<extra.testsource.dir>scala-2.11/src/test/scala</extra.testsource.dir>
|
||||
<extra.source.dir>scala-2.12/src/main/scala</extra.source.dir>
|
||||
<extra.testsource.dir>scala-2.12/src/test/scala</extra.testsource.dir>
|
||||
</properties>
|
||||
</profile>
|
||||
</profiles>
|
||||
-->
|
||||
|
||||
</project>
|
||||
|
|
|
@ -1,37 +0,0 @@
|
|||
/*
|
||||
* 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.repl
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
|
||||
object Main extends Logging {
|
||||
|
||||
initializeLogIfNecessary(true)
|
||||
Signaling.cancelOnInterrupt()
|
||||
|
||||
private var _interp: SparkILoop = _
|
||||
|
||||
def interp = _interp
|
||||
|
||||
def interp_=(i: SparkILoop) { _interp = i }
|
||||
|
||||
def main(args: Array[String]) {
|
||||
_interp = new SparkILoop
|
||||
_interp.process(args)
|
||||
}
|
||||
}
|
|
@ -1,46 +0,0 @@
|
|||
/*
|
||||
* 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.repl
|
||||
|
||||
import scala.tools.nsc.{CompilerCommand, Settings}
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi
|
||||
|
||||
/**
|
||||
* Command class enabling Spark-specific command line options (provided by
|
||||
* <i>org.apache.spark.repl.SparkRunnerSettings</i>).
|
||||
*
|
||||
* @example new SparkCommandLine(Nil).settings
|
||||
*
|
||||
* @param args The list of command line arguments
|
||||
* @param settings The underlying settings to associate with this set of
|
||||
* command-line options
|
||||
*/
|
||||
@DeveloperApi
|
||||
class SparkCommandLine(args: List[String], override val settings: Settings)
|
||||
extends CompilerCommand(args, settings) {
|
||||
def this(args: List[String], error: String => Unit) {
|
||||
this(args, new SparkRunnerSettings(error))
|
||||
}
|
||||
|
||||
def this(args: List[String]) {
|
||||
// scalastyle:off println
|
||||
this(args, str => Console.println("Error: " + str))
|
||||
// scalastyle:on println
|
||||
}
|
||||
}
|
|
@ -1,114 +0,0 @@
|
|||
// scalastyle:off
|
||||
|
||||
/* NSC -- new Scala compiler
|
||||
* Copyright 2005-2013 LAMP/EPFL
|
||||
* @author Paul Phillips
|
||||
*/
|
||||
|
||||
package org.apache.spark.repl
|
||||
|
||||
import scala.tools.nsc._
|
||||
import scala.tools.nsc.interpreter._
|
||||
|
||||
import scala.reflect.internal.util.BatchSourceFile
|
||||
import scala.tools.nsc.ast.parser.Tokens.EOF
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
|
||||
private[repl] trait SparkExprTyper extends Logging {
|
||||
val repl: SparkIMain
|
||||
|
||||
import repl._
|
||||
import global.{ reporter => _, Import => _, _ }
|
||||
import definitions._
|
||||
import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name }
|
||||
import naming.freshInternalVarName
|
||||
|
||||
object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] {
|
||||
def applyRule[T](code: String, rule: UnitParser => T): T = {
|
||||
reporter.reset()
|
||||
val scanner = newUnitParser(code)
|
||||
val result = rule(scanner)
|
||||
|
||||
if (!reporter.hasErrors)
|
||||
scanner.accept(EOF)
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
def defns(code: String) = stmts(code) collect { case x: DefTree => x }
|
||||
def expr(code: String) = applyRule(code, _.expr())
|
||||
def stmts(code: String) = applyRule(code, _.templateStats())
|
||||
def stmt(code: String) = stmts(code).last // guaranteed nonempty
|
||||
}
|
||||
|
||||
/** Parse a line into a sequence of trees. Returns None if the input is incomplete. */
|
||||
def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") {
|
||||
var isIncomplete = false
|
||||
reporter.withIncompleteHandler((_, _) => isIncomplete = true) {
|
||||
val trees = codeParser.stmts(line)
|
||||
if (reporter.hasErrors) {
|
||||
Some(Nil)
|
||||
} else if (isIncomplete) {
|
||||
None
|
||||
} else {
|
||||
Some(trees)
|
||||
}
|
||||
}
|
||||
}
|
||||
// def parsesAsExpr(line: String) = {
|
||||
// import codeParser._
|
||||
// (opt expr line).isDefined
|
||||
// }
|
||||
|
||||
def symbolOfLine(code: String): Symbol = {
|
||||
def asExpr(): Symbol = {
|
||||
val name = freshInternalVarName()
|
||||
// Typing it with a lazy val would give us the right type, but runs
|
||||
// into compiler bugs with things like existentials, so we compile it
|
||||
// behind a def and strip the NullaryMethodType which wraps the expr.
|
||||
val line = "def " + name + " = {\n" + code + "\n}"
|
||||
|
||||
interpretSynthetic(line) match {
|
||||
case IR.Success =>
|
||||
val sym0 = symbolOfTerm(name)
|
||||
// drop NullaryMethodType
|
||||
val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType)
|
||||
if (sym.info.typeSymbol eq UnitClass) NoSymbol else sym
|
||||
case _ => NoSymbol
|
||||
}
|
||||
}
|
||||
def asDefn(): Symbol = {
|
||||
val old = repl.definedSymbolList.toSet
|
||||
|
||||
interpretSynthetic(code) match {
|
||||
case IR.Success =>
|
||||
repl.definedSymbolList filterNot old match {
|
||||
case Nil => NoSymbol
|
||||
case sym :: Nil => sym
|
||||
case syms => NoSymbol.newOverloaded(NoPrefix, syms)
|
||||
}
|
||||
case _ => NoSymbol
|
||||
}
|
||||
}
|
||||
beQuietDuring(asExpr()) orElse beQuietDuring(asDefn())
|
||||
}
|
||||
|
||||
private var typeOfExpressionDepth = 0
|
||||
def typeOfExpression(expr: String, silent: Boolean = true): Type = {
|
||||
if (typeOfExpressionDepth > 2) {
|
||||
logDebug("Terminating typeOfExpression recursion for expression: " + expr)
|
||||
return NoType
|
||||
}
|
||||
typeOfExpressionDepth += 1
|
||||
// Don't presently have a good way to suppress undesirable success output
|
||||
// while letting errors through, so it is first trying it silently: if there
|
||||
// is an error, and errors are desired, then it re-evaluates non-silently
|
||||
// to induce the error message.
|
||||
try beSilentDuring(symbolOfLine(expr).tpe) match {
|
||||
case NoType if !silent => symbolOfLine(expr).tpe // generate error
|
||||
case tpe => tpe
|
||||
}
|
||||
finally typeOfExpressionDepth -= 1
|
||||
}
|
||||
}
|
|
@ -1,39 +0,0 @@
|
|||
/*
|
||||
* 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 scala.tools.nsc
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi
|
||||
|
||||
// NOTE: Forced to be public (and in scala.tools.nsc package) to access the
|
||||
// settings "explicitParentLoader" method
|
||||
|
||||
/**
|
||||
* Provides exposure for the explicitParentLoader method on settings instances.
|
||||
*/
|
||||
@DeveloperApi
|
||||
object SparkHelper {
|
||||
/**
|
||||
* Retrieves the explicit parent loader for the provided settings.
|
||||
*
|
||||
* @param settings The settings whose explicit parent loader to retrieve
|
||||
*
|
||||
* @return The Optional classloader representing the explicit parent loader
|
||||
*/
|
||||
@DeveloperApi
|
||||
def explicitParentLoader(settings: Settings) = settings.explicitParentLoader
|
||||
}
|
File diff suppressed because it is too large
Load diff
|
@ -1,168 +0,0 @@
|
|||
// scalastyle:off
|
||||
|
||||
/* NSC -- new Scala compiler
|
||||
* Copyright 2005-2013 LAMP/EPFL
|
||||
* @author Paul Phillips
|
||||
*/
|
||||
|
||||
package org.apache.spark.repl
|
||||
|
||||
import scala.tools.nsc._
|
||||
import scala.tools.nsc.interpreter._
|
||||
|
||||
import scala.tools.nsc.util.stackTraceString
|
||||
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
|
||||
/**
|
||||
* Machinery for the asynchronous initialization of the repl.
|
||||
*/
|
||||
private[repl] trait SparkILoopInit {
|
||||
self: SparkILoop =>
|
||||
|
||||
/** Print a welcome message */
|
||||
def printWelcome() {
|
||||
echo("""Welcome to
|
||||
____ __
|
||||
/ __/__ ___ _____/ /__
|
||||
_\ \/ _ \/ _ `/ __/ '_/
|
||||
/___/ .__/\_,_/_/ /_/\_\ version %s
|
||||
/_/
|
||||
""".format(SPARK_VERSION))
|
||||
import Properties._
|
||||
val welcomeMsg = "Using Scala %s (%s, Java %s)".format(
|
||||
versionString, javaVmName, javaVersion)
|
||||
echo(welcomeMsg)
|
||||
echo("Type in expressions to have them evaluated.")
|
||||
echo("Type :help for more information.")
|
||||
}
|
||||
|
||||
protected def asyncMessage(msg: String) {
|
||||
if (isReplInfo || isReplPower)
|
||||
echoAndRefresh(msg)
|
||||
}
|
||||
|
||||
private val initLock = new java.util.concurrent.locks.ReentrantLock()
|
||||
private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized
|
||||
private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized
|
||||
private val initStart = System.nanoTime
|
||||
|
||||
private def withLock[T](body: => T): T = {
|
||||
initLock.lock()
|
||||
try body
|
||||
finally initLock.unlock()
|
||||
}
|
||||
// a condition used to ensure serial access to the compiler.
|
||||
@volatile private var initIsComplete = false
|
||||
@volatile private var initError: String = null
|
||||
private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L)
|
||||
|
||||
// the method to be called when the interpreter is initialized.
|
||||
// Very important this method does nothing synchronous (i.e. do
|
||||
// not try to use the interpreter) because until it returns, the
|
||||
// repl's lazy val `global` is still locked.
|
||||
protected def initializedCallback() = withLock(initCompilerCondition.signal())
|
||||
|
||||
// Spins off a thread which awaits a single message once the interpreter
|
||||
// has been initialized.
|
||||
protected def createAsyncListener() = {
|
||||
io.spawn {
|
||||
withLock(initCompilerCondition.await())
|
||||
asyncMessage("[info] compiler init time: " + elapsed() + " s.")
|
||||
postInitialization()
|
||||
}
|
||||
}
|
||||
|
||||
// called from main repl loop
|
||||
protected def awaitInitialized(): Boolean = {
|
||||
if (!initIsComplete)
|
||||
withLock { while (!initIsComplete) initLoopCondition.await() }
|
||||
if (initError != null) {
|
||||
// scalastyle:off println
|
||||
println("""
|
||||
|Failed to initialize the REPL due to an unexpected error.
|
||||
|This is a bug, please, report it along with the error diagnostics printed below.
|
||||
|%s.""".stripMargin.format(initError)
|
||||
)
|
||||
// scalastyle:on println
|
||||
false
|
||||
} else true
|
||||
}
|
||||
// private def warningsThunks = List(
|
||||
// () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _),
|
||||
// )
|
||||
|
||||
protected def postInitThunks = List[Option[() => Unit]](
|
||||
Some(intp.setContextClassLoader _),
|
||||
if (isReplPower) Some(() => enablePowerMode(true)) else None
|
||||
).flatten
|
||||
// ++ (
|
||||
// warningsThunks
|
||||
// )
|
||||
// called once after init condition is signalled
|
||||
protected def postInitialization() {
|
||||
try {
|
||||
postInitThunks foreach (f => addThunk(f()))
|
||||
runThunks()
|
||||
} catch {
|
||||
case ex: Throwable =>
|
||||
initError = stackTraceString(ex)
|
||||
throw ex
|
||||
} finally {
|
||||
initIsComplete = true
|
||||
|
||||
if (isAsync) {
|
||||
asyncMessage("[info] total init time: " + elapsed() + " s.")
|
||||
withLock(initLoopCondition.signal())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def initializeSpark() {
|
||||
intp.beQuietDuring {
|
||||
command("""
|
||||
@transient val spark = org.apache.spark.repl.Main.interp.createSparkSession()
|
||||
@transient val sc = {
|
||||
val _sc = spark.sparkContext
|
||||
if (_sc.getConf.getBoolean("spark.ui.reverseProxy", false)) {
|
||||
val proxyUrl = _sc.getConf.get("spark.ui.reverseProxyUrl", null)
|
||||
if (proxyUrl != null) {
|
||||
println(s"Spark Context Web UI is available at ${proxyUrl}/proxy/${_sc.applicationId}")
|
||||
} else {
|
||||
println(s"Spark Context Web UI is available at Spark Master Public URL")
|
||||
}
|
||||
} else {
|
||||
_sc.uiWebUrl.foreach {
|
||||
webUrl => println(s"Spark context Web UI available at ${webUrl}")
|
||||
}
|
||||
}
|
||||
println("Spark context available as 'sc' " +
|
||||
s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
|
||||
println("Spark session available as 'spark'.")
|
||||
_sc
|
||||
}
|
||||
""")
|
||||
command("import org.apache.spark.SparkContext._")
|
||||
command("import spark.implicits._")
|
||||
command("import spark.sql")
|
||||
command("import org.apache.spark.sql.functions._")
|
||||
}
|
||||
}
|
||||
|
||||
// code to be executed only after the interpreter is initialized
|
||||
// and the lazy val `global` can be accessed without risk of deadlock.
|
||||
private var pendingThunks: List[() => Unit] = Nil
|
||||
protected def addThunk(body: => Unit) = synchronized {
|
||||
pendingThunks :+= (() => body)
|
||||
}
|
||||
protected def runThunks(): Unit = synchronized {
|
||||
if (pendingThunks.nonEmpty)
|
||||
logDebug("Clearing " + pendingThunks.size + " thunks.")
|
||||
|
||||
while (pendingThunks.nonEmpty) {
|
||||
val thunk = pendingThunks.head
|
||||
pendingThunks = pendingThunks.tail
|
||||
thunk()
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load diff
|
@ -1,239 +0,0 @@
|
|||
// scalastyle:off
|
||||
|
||||
/* NSC -- new Scala compiler
|
||||
* Copyright 2005-2013 LAMP/EPFL
|
||||
* @author Paul Phillips
|
||||
*/
|
||||
|
||||
package org.apache.spark.repl
|
||||
|
||||
import scala.tools.nsc._
|
||||
import scala.tools.nsc.interpreter._
|
||||
|
||||
import scala.collection.{ mutable, immutable }
|
||||
|
||||
private[repl] trait SparkImports {
|
||||
self: SparkIMain =>
|
||||
|
||||
import global._
|
||||
import definitions.{ ScalaPackage, JavaLangPackage, PredefModule }
|
||||
import memberHandlers._
|
||||
|
||||
def isNoImports = settings.noimports.value
|
||||
def isNoPredef = settings.nopredef.value
|
||||
|
||||
/** Synthetic import handlers for the language defined imports. */
|
||||
private def makeWildcardImportHandler(sym: Symbol): ImportHandler = {
|
||||
val hd :: tl = sym.fullName.split('.').toList map newTermName
|
||||
val tree = Import(
|
||||
tl.foldLeft(Ident(hd): Tree)((x, y) => Select(x, y)),
|
||||
ImportSelector.wildList
|
||||
)
|
||||
tree setSymbol sym
|
||||
new ImportHandler(tree)
|
||||
}
|
||||
|
||||
/** Symbols whose contents are language-defined to be imported. */
|
||||
def languageWildcardSyms: List[Symbol] = List(JavaLangPackage, ScalaPackage, PredefModule)
|
||||
def languageWildcards: List[Type] = languageWildcardSyms map (_.tpe)
|
||||
def languageWildcardHandlers = languageWildcardSyms map makeWildcardImportHandler
|
||||
|
||||
def allImportedNames = importHandlers flatMap (_.importedNames)
|
||||
def importedTerms = onlyTerms(allImportedNames)
|
||||
def importedTypes = onlyTypes(allImportedNames)
|
||||
|
||||
/** Types which have been wildcard imported, such as:
|
||||
* val x = "abc" ; import x._ // type java.lang.String
|
||||
* import java.lang.String._ // object java.lang.String
|
||||
*
|
||||
* Used by tab completion.
|
||||
*
|
||||
* XXX right now this gets import x._ and import java.lang.String._,
|
||||
* but doesn't figure out import String._. There's a lot of ad hoc
|
||||
* scope twiddling which should be swept away in favor of digging
|
||||
* into the compiler scopes.
|
||||
*/
|
||||
def sessionWildcards: List[Type] = {
|
||||
importHandlers filter (_.importsWildcard) map (_.targetType) distinct
|
||||
}
|
||||
def wildcardTypes = languageWildcards ++ sessionWildcards
|
||||
|
||||
def languageSymbols = languageWildcardSyms flatMap membersAtPickler
|
||||
def sessionImportedSymbols = importHandlers flatMap (_.importedSymbols)
|
||||
def importedSymbols = languageSymbols ++ sessionImportedSymbols
|
||||
def importedTermSymbols = importedSymbols collect { case x: TermSymbol => x }
|
||||
def importedTypeSymbols = importedSymbols collect { case x: TypeSymbol => x }
|
||||
def implicitSymbols = importedSymbols filter (_.isImplicit)
|
||||
|
||||
def importedTermNamed(name: String): Symbol =
|
||||
importedTermSymbols find (_.name.toString == name) getOrElse NoSymbol
|
||||
|
||||
/** Tuples of (source, imported symbols) in the order they were imported.
|
||||
*/
|
||||
def importedSymbolsBySource: List[(Symbol, List[Symbol])] = {
|
||||
val lang = languageWildcardSyms map (sym => (sym, membersAtPickler(sym)))
|
||||
val session = importHandlers filter (_.targetType != NoType) map { mh =>
|
||||
(mh.targetType.typeSymbol, mh.importedSymbols)
|
||||
}
|
||||
|
||||
lang ++ session
|
||||
}
|
||||
def implicitSymbolsBySource: List[(Symbol, List[Symbol])] = {
|
||||
importedSymbolsBySource map {
|
||||
case (k, vs) => (k, vs filter (_.isImplicit))
|
||||
} filterNot (_._2.isEmpty)
|
||||
}
|
||||
|
||||
/** Compute imports that allow definitions from previous
|
||||
* requests to be visible in a new request. Returns
|
||||
* three pieces of related code:
|
||||
*
|
||||
* 1. An initial code fragment that should go before
|
||||
* the code of the new request.
|
||||
*
|
||||
* 2. A code fragment that should go after the code
|
||||
* of the new request.
|
||||
*
|
||||
* 3. An access path which can be traversed to access
|
||||
* any bindings inside code wrapped by #1 and #2 .
|
||||
*
|
||||
* The argument is a set of Names that need to be imported.
|
||||
*
|
||||
* Limitations: This method is not as precise as it could be.
|
||||
* (1) It does not process wildcard imports to see what exactly
|
||||
* they import.
|
||||
* (2) If it imports any names from a request, it imports all
|
||||
* of them, which is not really necessary.
|
||||
* (3) It imports multiple same-named implicits, but only the
|
||||
* last one imported is actually usable.
|
||||
*/
|
||||
case class SparkComputedImports(prepend: String, append: String, access: String)
|
||||
def fallback = System.getProperty("spark.repl.fallback", "false").toBoolean
|
||||
|
||||
protected def importsCode(wanted: Set[Name], definedClass: Boolean): SparkComputedImports = {
|
||||
/** Narrow down the list of requests from which imports
|
||||
* should be taken. Removes requests which cannot contribute
|
||||
* useful imports for the specified set of wanted names.
|
||||
*/
|
||||
case class ReqAndHandler(req: Request, handler: MemberHandler) { }
|
||||
|
||||
def reqsToUse: List[ReqAndHandler] = {
|
||||
/**
|
||||
* Loop through a list of MemberHandlers and select which ones to keep.
|
||||
* 'wanted' is the set of names that need to be imported.
|
||||
*/
|
||||
def select(reqs: List[ReqAndHandler], wanted: Set[Name]): List[ReqAndHandler] = {
|
||||
// Single symbol imports might be implicits! See bug #1752. Rather than
|
||||
// try to finesse this, we will mimic all imports for now.
|
||||
def keepHandler(handler: MemberHandler) = handler match {
|
||||
/* This case clause tries to "precisely" import only what is required. And in this
|
||||
* it may miss out on some implicits, because implicits are not known in `wanted`. Thus
|
||||
* it is suitable for defining classes. AFAIK while defining classes implicits are not
|
||||
* needed.*/
|
||||
case h: ImportHandler if definedClass && !fallback =>
|
||||
h.importedNames.exists(x => wanted.contains(x))
|
||||
case _: ImportHandler => true
|
||||
case x => x.definesImplicit || (x.definedNames exists wanted)
|
||||
}
|
||||
|
||||
reqs match {
|
||||
case Nil => Nil
|
||||
case rh :: rest if !keepHandler(rh.handler) => select(rest, wanted)
|
||||
case rh :: rest =>
|
||||
import rh.handler._
|
||||
val newWanted = wanted ++ referencedNames -- definedNames -- importedNames
|
||||
rh :: select(rest, newWanted)
|
||||
}
|
||||
}
|
||||
|
||||
/** Flatten the handlers out and pair each with the original request */
|
||||
select(allReqAndHandlers reverseMap { case (r, h) => ReqAndHandler(r, h) }, wanted).reverse
|
||||
}
|
||||
|
||||
val code, trailingBraces, accessPath = new StringBuilder
|
||||
val currentImps = mutable.HashSet[Name]()
|
||||
|
||||
// add code for a new object to hold some imports
|
||||
def addWrapper() {
|
||||
val impname = nme.INTERPRETER_IMPORT_WRAPPER
|
||||
code append "class %sC extends Serializable {\n".format(impname)
|
||||
trailingBraces append "}\nval " + impname + " = new " + impname + "C;\n"
|
||||
accessPath append ("." + impname)
|
||||
|
||||
currentImps.clear
|
||||
// code append "object %s {\n".format(impname)
|
||||
// trailingBraces append "}\n"
|
||||
// accessPath append ("." + impname)
|
||||
|
||||
// currentImps.clear
|
||||
}
|
||||
|
||||
addWrapper()
|
||||
|
||||
// loop through previous requests, adding imports for each one
|
||||
for (ReqAndHandler(req, handler) <- reqsToUse) {
|
||||
handler match {
|
||||
// If the user entered an import, then just use it; add an import wrapping
|
||||
// level if the import might conflict with some other import
|
||||
case x: ImportHandler =>
|
||||
if (x.importsWildcard || currentImps.exists(x.importedNames contains _))
|
||||
addWrapper()
|
||||
|
||||
code append (x.member + "\n")
|
||||
|
||||
// give wildcard imports a import wrapper all to their own
|
||||
if (x.importsWildcard) addWrapper()
|
||||
else currentImps ++= x.importedNames
|
||||
|
||||
// For other requests, import each defined name.
|
||||
// import them explicitly instead of with _, so that
|
||||
// ambiguity errors will not be generated. Also, quote
|
||||
// the name of the variable, so that we don't need to
|
||||
// handle quoting keywords separately.
|
||||
case x: ClassHandler if !fallback =>
|
||||
// I am trying to guess if the import is a defined class
|
||||
// This is an ugly hack, I am not 100% sure of the consequences.
|
||||
// Here we, let everything but "defined classes" use the import with val.
|
||||
// The reason for this is, otherwise the remote executor tries to pull the
|
||||
// classes involved and may fail.
|
||||
for (imv <- x.definedNames) {
|
||||
val objName = req.lineRep.readPath
|
||||
code.append("import " + objName + ".INSTANCE" + req.accessPath + ".`" + imv + "`\n")
|
||||
}
|
||||
|
||||
case x =>
|
||||
for (imv <- x.definedNames) {
|
||||
if (currentImps contains imv) addWrapper()
|
||||
val objName = req.lineRep.readPath
|
||||
val valName = "$VAL" + newValId()
|
||||
|
||||
if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported
|
||||
code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
|
||||
code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
|
||||
}
|
||||
// code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
|
||||
// code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
|
||||
// code append ("import " + (req fullPath imv) + "\n")
|
||||
currentImps += imv
|
||||
}
|
||||
}
|
||||
}
|
||||
// add one extra wrapper, to prevent warnings in the common case of
|
||||
// redefining the value bound in the last interpreter request.
|
||||
addWrapper()
|
||||
SparkComputedImports(code.toString, trailingBraces.toString, accessPath.toString)
|
||||
}
|
||||
|
||||
private def allReqAndHandlers =
|
||||
prevRequestList flatMap (req => req.handlers map (req -> _))
|
||||
|
||||
private def membersAtPickler(sym: Symbol): List[Symbol] =
|
||||
beforePickler(sym.info.nonPrivateMembers.toList)
|
||||
|
||||
private var curValId = 0
|
||||
|
||||
private def newValId(): Int = {
|
||||
curValId += 1
|
||||
curValId
|
||||
}
|
||||
}
|
|
@ -1,403 +0,0 @@
|
|||
// scalastyle:off
|
||||
|
||||
/* NSC -- new Scala compiler
|
||||
* Copyright 2005-2013 LAMP/EPFL
|
||||
* @author Paul Phillips
|
||||
*/
|
||||
|
||||
package org.apache.spark.repl
|
||||
|
||||
import scala.tools.nsc._
|
||||
import scala.tools.nsc.interpreter._
|
||||
|
||||
import scala.tools.jline._
|
||||
import scala.tools.jline.console.completer._
|
||||
import Completion._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi
|
||||
import org.apache.spark.internal.Logging
|
||||
|
||||
/**
|
||||
* Represents an auto-completion tool for the supplied interpreter that
|
||||
* utilizes supplied queries for valid completions based on the current
|
||||
* contents of the internal buffer.
|
||||
*
|
||||
* @param intp The interpreter to use for information retrieval to do with
|
||||
* auto completion
|
||||
*/
|
||||
@DeveloperApi
|
||||
class SparkJLineCompletion(val intp: SparkIMain) extends Completion with CompletionOutput with Logging {
|
||||
// NOTE: Exposed in package as used in quite a few classes
|
||||
// NOTE: Must be public to override the global found in CompletionOutput
|
||||
val global: intp.global.type = intp.global
|
||||
|
||||
import global._
|
||||
import definitions.{ PredefModule, AnyClass, AnyRefClass, ScalaPackage, JavaLangPackage }
|
||||
import rootMirror.{ RootClass, getModuleIfDefined }
|
||||
type ExecResult = Any
|
||||
import intp.{ debugging }
|
||||
|
||||
/**
|
||||
* Represents the level of verbosity. Increments with consecutive tabs.
|
||||
*/
|
||||
@DeveloperApi
|
||||
var verbosity: Int = 0
|
||||
|
||||
/**
|
||||
* Resets the level of verbosity to zero.
|
||||
*/
|
||||
@DeveloperApi
|
||||
def resetVerbosity() = verbosity = 0
|
||||
|
||||
private def getSymbol(name: String, isModule: Boolean) = (
|
||||
if (isModule) getModuleIfDefined(name)
|
||||
else getModuleIfDefined(name)
|
||||
)
|
||||
private def getType(name: String, isModule: Boolean) = getSymbol(name, isModule).tpe
|
||||
private def typeOf(name: String) = getType(name, false)
|
||||
private def moduleOf(name: String) = getType(name, true)
|
||||
|
||||
trait CompilerCompletion {
|
||||
def tp: Type
|
||||
def effectiveTp = tp match {
|
||||
case MethodType(Nil, resType) => resType
|
||||
case NullaryMethodType(resType) => resType
|
||||
case _ => tp
|
||||
}
|
||||
|
||||
// for some reason any's members don't show up in subclasses, which
|
||||
// we need so 5.<tab> offers asInstanceOf etc.
|
||||
private def anyMembers = AnyClass.tpe.nonPrivateMembers
|
||||
def anyRefMethodsToShow = Set("isInstanceOf", "asInstanceOf", "toString")
|
||||
|
||||
def tos(sym: Symbol): String = sym.decodedName
|
||||
def memberNamed(s: String) = afterTyper(effectiveTp member newTermName(s))
|
||||
def hasMethod(s: String) = memberNamed(s).isMethod
|
||||
|
||||
// XXX we'd like to say "filterNot (_.isDeprecated)" but this causes the
|
||||
// compiler to crash for reasons not yet known.
|
||||
def members = afterTyper((effectiveTp.nonPrivateMembers.toList ++ anyMembers) filter (_.isPublic))
|
||||
def methods = members.toList filter (_.isMethod)
|
||||
def packages = members.toList filter (_.isPackage)
|
||||
def aliases = members.toList filter (_.isAliasType)
|
||||
|
||||
def memberNames = members map tos
|
||||
def methodNames = methods map tos
|
||||
def packageNames = packages map tos
|
||||
def aliasNames = aliases map tos
|
||||
}
|
||||
|
||||
object NoTypeCompletion extends TypeMemberCompletion(NoType) {
|
||||
override def memberNamed(s: String) = NoSymbol
|
||||
override def members = Nil
|
||||
override def follow(s: String) = None
|
||||
override def alternativesFor(id: String) = Nil
|
||||
}
|
||||
|
||||
object TypeMemberCompletion {
|
||||
def apply(tp: Type, runtimeType: Type, param: NamedParam): TypeMemberCompletion = {
|
||||
new TypeMemberCompletion(tp) {
|
||||
var upgraded = false
|
||||
lazy val upgrade = {
|
||||
intp rebind param
|
||||
intp.reporter.printMessage("\nRebinding stable value %s from %s to %s".format(param.name, tp, param.tpe))
|
||||
upgraded = true
|
||||
new TypeMemberCompletion(runtimeType)
|
||||
}
|
||||
override def completions(verbosity: Int) = {
|
||||
super.completions(verbosity) ++ (
|
||||
if (verbosity == 0) Nil
|
||||
else upgrade.completions(verbosity)
|
||||
)
|
||||
}
|
||||
override def follow(s: String) = super.follow(s) orElse {
|
||||
if (upgraded) upgrade.follow(s)
|
||||
else None
|
||||
}
|
||||
override def alternativesFor(id: String) = super.alternativesFor(id) ++ (
|
||||
if (upgraded) upgrade.alternativesFor(id)
|
||||
else Nil
|
||||
) distinct
|
||||
}
|
||||
}
|
||||
def apply(tp: Type): TypeMemberCompletion = {
|
||||
if (tp eq NoType) NoTypeCompletion
|
||||
else if (tp.typeSymbol.isPackageClass) new PackageCompletion(tp)
|
||||
else new TypeMemberCompletion(tp)
|
||||
}
|
||||
def imported(tp: Type) = new ImportCompletion(tp)
|
||||
}
|
||||
|
||||
class TypeMemberCompletion(val tp: Type) extends CompletionAware
|
||||
with CompilerCompletion {
|
||||
def excludeEndsWith: List[String] = Nil
|
||||
def excludeStartsWith: List[String] = List("<") // <byname>, <repeated>, etc.
|
||||
def excludeNames: List[String] = (anyref.methodNames filterNot anyRefMethodsToShow) :+ "_root_"
|
||||
|
||||
def methodSignatureString(sym: Symbol) = {
|
||||
IMain stripString afterTyper(new MethodSymbolOutput(sym).methodString())
|
||||
}
|
||||
|
||||
def exclude(name: String): Boolean = (
|
||||
(name contains "$") ||
|
||||
(excludeNames contains name) ||
|
||||
(excludeEndsWith exists (name endsWith _)) ||
|
||||
(excludeStartsWith exists (name startsWith _))
|
||||
)
|
||||
def filtered(xs: List[String]) = xs filterNot exclude distinct
|
||||
|
||||
def completions(verbosity: Int) =
|
||||
debugging(tp + " completions ==> ")(filtered(memberNames))
|
||||
|
||||
override def follow(s: String): Option[CompletionAware] =
|
||||
debugging(tp + " -> '" + s + "' ==> ")(Some(TypeMemberCompletion(memberNamed(s).tpe)) filterNot (_ eq NoTypeCompletion))
|
||||
|
||||
override def alternativesFor(id: String): List[String] =
|
||||
debugging(id + " alternatives ==> ") {
|
||||
val alts = members filter (x => x.isMethod && tos(x) == id) map methodSignatureString
|
||||
|
||||
if (alts.nonEmpty) "" :: alts else Nil
|
||||
}
|
||||
|
||||
override def toString = "%s (%d members)".format(tp, members.size)
|
||||
}
|
||||
|
||||
class PackageCompletion(tp: Type) extends TypeMemberCompletion(tp) {
|
||||
override def excludeNames = anyref.methodNames
|
||||
}
|
||||
|
||||
class LiteralCompletion(lit: Literal) extends TypeMemberCompletion(lit.value.tpe) {
|
||||
override def completions(verbosity: Int) = verbosity match {
|
||||
case 0 => filtered(memberNames)
|
||||
case _ => memberNames
|
||||
}
|
||||
}
|
||||
|
||||
class ImportCompletion(tp: Type) extends TypeMemberCompletion(tp) {
|
||||
override def completions(verbosity: Int) = verbosity match {
|
||||
case 0 => filtered(members filterNot (_.isSetter) map tos)
|
||||
case _ => super.completions(verbosity)
|
||||
}
|
||||
}
|
||||
|
||||
// not for completion but for excluding
|
||||
object anyref extends TypeMemberCompletion(AnyRefClass.tpe) { }
|
||||
|
||||
// the unqualified vals/defs/etc visible in the repl
|
||||
object ids extends CompletionAware {
|
||||
override def completions(verbosity: Int) = intp.unqualifiedIds ++ List("classOf") //, "_root_")
|
||||
// now we use the compiler for everything.
|
||||
override def follow(id: String): Option[CompletionAware] = {
|
||||
if (!completions(0).contains(id))
|
||||
return None
|
||||
|
||||
val tpe = intp typeOfExpression id
|
||||
if (tpe == NoType)
|
||||
return None
|
||||
|
||||
def default = Some(TypeMemberCompletion(tpe))
|
||||
|
||||
// only rebinding vals in power mode for now.
|
||||
if (!isReplPower) default
|
||||
else intp runtimeClassAndTypeOfTerm id match {
|
||||
case Some((clazz, runtimeType)) =>
|
||||
val sym = intp.symbolOfTerm(id)
|
||||
if (sym.isStable) {
|
||||
val param = new NamedParam.Untyped(id, intp valueOfTerm id getOrElse null)
|
||||
Some(TypeMemberCompletion(tpe, runtimeType, param))
|
||||
}
|
||||
else default
|
||||
case _ =>
|
||||
default
|
||||
}
|
||||
}
|
||||
override def toString = "<repl ids> (%s)".format(completions(0).size)
|
||||
}
|
||||
|
||||
// user-issued wildcard imports like "import global._" or "import String._"
|
||||
private def imported = intp.sessionWildcards map TypeMemberCompletion.imported
|
||||
|
||||
// literal Ints, Strings, etc.
|
||||
object literals extends CompletionAware {
|
||||
def simpleParse(code: String): Tree = newUnitParser(code).templateStats().last
|
||||
def completions(verbosity: Int) = Nil
|
||||
|
||||
override def follow(id: String) = simpleParse(id) match {
|
||||
case x: Literal => Some(new LiteralCompletion(x))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
// top level packages
|
||||
object rootClass extends TypeMemberCompletion(RootClass.tpe) {
|
||||
override def completions(verbosity: Int) = super.completions(verbosity) :+ "_root_"
|
||||
override def follow(id: String) = id match {
|
||||
case "_root_" => Some(this)
|
||||
case _ => super.follow(id)
|
||||
}
|
||||
}
|
||||
// members of Predef
|
||||
object predef extends TypeMemberCompletion(PredefModule.tpe) {
|
||||
override def excludeEndsWith = super.excludeEndsWith ++ List("Wrapper", "ArrayOps")
|
||||
override def excludeStartsWith = super.excludeStartsWith ++ List("wrap")
|
||||
override def excludeNames = anyref.methodNames
|
||||
|
||||
override def exclude(name: String) = super.exclude(name) || (
|
||||
(name contains "2")
|
||||
)
|
||||
|
||||
override def completions(verbosity: Int) = verbosity match {
|
||||
case 0 => Nil
|
||||
case _ => super.completions(verbosity)
|
||||
}
|
||||
}
|
||||
// members of scala.*
|
||||
object scalalang extends PackageCompletion(ScalaPackage.tpe) {
|
||||
def arityClasses = List("Product", "Tuple", "Function")
|
||||
def skipArity(name: String) = arityClasses exists (x => name != x && (name startsWith x))
|
||||
override def exclude(name: String) = super.exclude(name) || (
|
||||
skipArity(name)
|
||||
)
|
||||
|
||||
override def completions(verbosity: Int) = verbosity match {
|
||||
case 0 => filtered(packageNames ++ aliasNames)
|
||||
case _ => super.completions(verbosity)
|
||||
}
|
||||
}
|
||||
// members of java.lang.*
|
||||
object javalang extends PackageCompletion(JavaLangPackage.tpe) {
|
||||
override lazy val excludeEndsWith = super.excludeEndsWith ++ List("Exception", "Error")
|
||||
override lazy val excludeStartsWith = super.excludeStartsWith ++ List("CharacterData")
|
||||
|
||||
override def completions(verbosity: Int) = verbosity match {
|
||||
case 0 => filtered(packageNames)
|
||||
case _ => super.completions(verbosity)
|
||||
}
|
||||
}
|
||||
|
||||
// the list of completion aware objects which should be consulted
|
||||
// for top level unqualified, it's too noisy to let much in.
|
||||
private lazy val topLevelBase: List[CompletionAware] = List(ids, rootClass, predef, scalalang, javalang, literals)
|
||||
private def topLevel = topLevelBase ++ imported
|
||||
private def topLevelThreshold = 50
|
||||
|
||||
// the first tier of top level objects (doesn't include file completion)
|
||||
private def topLevelFor(parsed: Parsed): List[String] = {
|
||||
val buf = new ListBuffer[String]
|
||||
topLevel foreach { ca =>
|
||||
buf ++= (ca completionsFor parsed)
|
||||
|
||||
if (buf.size > topLevelThreshold)
|
||||
return buf.toList.sorted
|
||||
}
|
||||
buf.toList
|
||||
}
|
||||
|
||||
// the most recent result
|
||||
private def lastResult = Forwarder(() => ids follow intp.mostRecentVar)
|
||||
|
||||
private def lastResultFor(parsed: Parsed) = {
|
||||
/** The logic is a little tortured right now because normally '.' is
|
||||
* ignored as a delimiter, but on .<tab> it needs to be propagated.
|
||||
*/
|
||||
val xs = lastResult completionsFor parsed
|
||||
if (parsed.isEmpty) xs map ("." + _) else xs
|
||||
}
|
||||
|
||||
// generic interface for querying (e.g. interpreter loop, testing)
|
||||
private def completions(buf: String): List[String] =
|
||||
topLevelFor(Parsed.dotted(buf + ".", buf.length + 1))
|
||||
|
||||
/**
|
||||
* Constructs a new ScalaCompleter for auto completion.
|
||||
*
|
||||
* @return The new JLineTabCompletion instance
|
||||
*/
|
||||
@DeveloperApi
|
||||
def completer(): ScalaCompleter = new JLineTabCompletion
|
||||
|
||||
/** This gets a little bit hairy. It's no small feat delegating everything
|
||||
* and also keeping track of exactly where the cursor is and where it's supposed
|
||||
* to end up. The alternatives mechanism is a little hacky: if there is an empty
|
||||
* string in the list of completions, that means we are expanding a unique
|
||||
* completion, so don't update the "last" buffer because it'll be wrong.
|
||||
*/
|
||||
class JLineTabCompletion extends ScalaCompleter {
|
||||
// For recording the buffer on the last tab hit
|
||||
private var lastBuf: String = ""
|
||||
private var lastCursor: Int = -1
|
||||
|
||||
// Does this represent two consecutive tabs?
|
||||
def isConsecutiveTabs(buf: String, cursor: Int) =
|
||||
cursor == lastCursor && buf == lastBuf
|
||||
|
||||
// Longest common prefix
|
||||
def commonPrefix(xs: List[String]): String = {
|
||||
if (xs.isEmpty || xs.contains("")) ""
|
||||
else xs.head.head match {
|
||||
case ch =>
|
||||
if (xs.tail forall (_.head == ch)) "" + ch + commonPrefix(xs map (_.tail))
|
||||
else ""
|
||||
}
|
||||
}
|
||||
|
||||
// This is jline's entry point for completion.
|
||||
override def complete(buf: String, cursor: Int): Candidates = {
|
||||
verbosity = if (isConsecutiveTabs(buf, cursor)) verbosity + 1 else 0
|
||||
logDebug("\ncomplete(%s, %d) last = (%s, %d), verbosity: %s".format(buf, cursor, lastBuf, lastCursor, verbosity))
|
||||
|
||||
// we don't try lower priority completions unless higher ones return no results.
|
||||
def tryCompletion(p: Parsed, completionFunction: Parsed => List[String]): Option[Candidates] = {
|
||||
val winners = completionFunction(p)
|
||||
if (winners.isEmpty)
|
||||
return None
|
||||
val newCursor =
|
||||
if (winners contains "") p.cursor
|
||||
else {
|
||||
val advance = commonPrefix(winners)
|
||||
lastCursor = p.position + advance.length
|
||||
lastBuf = (buf take p.position) + advance
|
||||
logDebug("tryCompletion(%s, _) lastBuf = %s, lastCursor = %s, p.position = %s".format(
|
||||
p, lastBuf, lastCursor, p.position))
|
||||
p.position
|
||||
}
|
||||
|
||||
Some(Candidates(newCursor, winners))
|
||||
}
|
||||
|
||||
def mkDotted = Parsed.dotted(buf, cursor) withVerbosity verbosity
|
||||
def mkUndelimited = Parsed.undelimited(buf, cursor) withVerbosity verbosity
|
||||
|
||||
// a single dot is special cased to completion on the previous result
|
||||
def lastResultCompletion =
|
||||
if (!looksLikeInvocation(buf)) None
|
||||
else tryCompletion(Parsed.dotted(buf drop 1, cursor), lastResultFor)
|
||||
|
||||
def tryAll = (
|
||||
lastResultCompletion
|
||||
orElse tryCompletion(mkDotted, topLevelFor)
|
||||
getOrElse Candidates(cursor, Nil)
|
||||
)
|
||||
|
||||
/**
|
||||
* This is the kickoff point for all manner of theoretically
|
||||
* possible compiler unhappiness. The fault may be here or
|
||||
* elsewhere, but we don't want to crash the repl regardless.
|
||||
* The compiler makes it impossible to avoid catching Throwable
|
||||
* with its unfortunate tendency to throw java.lang.Errors and
|
||||
* AssertionErrors as the hats drop. We take two swings at it
|
||||
* because there are some spots which like to throw an assertion
|
||||
* once, then work after that. Yeah, what can I say.
|
||||
*/
|
||||
try tryAll
|
||||
catch { case ex: Throwable =>
|
||||
logWarning("Error: complete(%s, %s) provoked".format(buf, cursor) + ex)
|
||||
Candidates(cursor,
|
||||
if (isReplDebug) List("<error:" + ex + ">")
|
||||
else Nil
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,90 +0,0 @@
|
|||
// scalastyle:off
|
||||
|
||||
/* NSC -- new Scala compiler
|
||||
* Copyright 2005-2013 LAMP/EPFL
|
||||
* @author Stepan Koltsov
|
||||
*/
|
||||
|
||||
package org.apache.spark.repl
|
||||
|
||||
import scala.reflect.io.{Path, File}
|
||||
import scala.tools.nsc._
|
||||
import scala.tools.nsc.interpreter._
|
||||
import scala.tools.nsc.interpreter.session.JLineHistory.JLineFileHistory
|
||||
|
||||
import scala.tools.jline.console.ConsoleReader
|
||||
import scala.tools.jline.console.completer._
|
||||
import session._
|
||||
import scala.collection.JavaConverters._
|
||||
import Completion._
|
||||
import io.Streamable.slurp
|
||||
|
||||
/**
|
||||
* Reads from the console using JLine.
|
||||
*/
|
||||
private[repl] class SparkJLineReader(_completion: => Completion) extends InteractiveReader {
|
||||
val interactive = true
|
||||
val consoleReader = new JLineConsoleReader()
|
||||
|
||||
lazy val completion = _completion
|
||||
lazy val history: JLineHistory = new SparkJLineHistory
|
||||
|
||||
private def term = consoleReader.getTerminal()
|
||||
def reset() = term.reset()
|
||||
def init() = term.init()
|
||||
|
||||
def scalaToJline(tc: ScalaCompleter): Completer = new Completer {
|
||||
def complete(_buf: String, cursor: Int, candidates: JList[CharSequence]): Int = {
|
||||
val buf = if (_buf == null) "" else _buf
|
||||
val Candidates(newCursor, newCandidates) = tc.complete(buf, cursor)
|
||||
newCandidates foreach (candidates add _)
|
||||
newCursor
|
||||
}
|
||||
}
|
||||
|
||||
class JLineConsoleReader extends ConsoleReader with ConsoleReaderHelper {
|
||||
if ((history: History) ne NoHistory)
|
||||
this setHistory history
|
||||
|
||||
// working around protected/trait/java insufficiencies.
|
||||
def goBack(num: Int): Unit = back(num)
|
||||
def readOneKey(prompt: String) = {
|
||||
this.print(prompt)
|
||||
this.flush()
|
||||
this.readVirtualKey()
|
||||
}
|
||||
def eraseLine() = consoleReader.resetPromptLine("", "", 0)
|
||||
def redrawLineAndFlush(): Unit = { flush() ; drawLine() ; flush() }
|
||||
// override def readLine(prompt: String): String
|
||||
|
||||
// A hook for running code after the repl is done initializing.
|
||||
lazy val postInit: Unit = {
|
||||
this setBellEnabled false
|
||||
|
||||
if (completion ne NoCompletion) {
|
||||
val argCompletor: ArgumentCompleter =
|
||||
new ArgumentCompleter(new JLineDelimiter, scalaToJline(completion.completer()))
|
||||
argCompletor setStrict false
|
||||
|
||||
this addCompleter argCompletor
|
||||
this setAutoprintThreshold 400 // max completion candidates without warning
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def currentLine = consoleReader.getCursorBuffer.buffer.toString
|
||||
def redrawLine() = consoleReader.redrawLineAndFlush()
|
||||
def eraseLine() = consoleReader.eraseLine()
|
||||
// Alternate implementation, not sure if/when I need this.
|
||||
// def eraseLine() = while (consoleReader.delete()) { }
|
||||
def readOneLine(prompt: String) = consoleReader readLine prompt
|
||||
def readOneKey(prompt: String) = consoleReader readOneKey prompt
|
||||
}
|
||||
|
||||
/** Changes the default history file to not collide with the scala repl's. */
|
||||
private[repl] class SparkJLineHistory extends JLineFileHistory {
|
||||
import Properties.userHome
|
||||
|
||||
def defaultFileName = ".spark_history"
|
||||
override protected lazy val historyFile = File(Path(userHome) / defaultFileName)
|
||||
}
|
|
@ -1,232 +0,0 @@
|
|||
// scalastyle:off
|
||||
|
||||
/* NSC -- new Scala compiler
|
||||
* Copyright 2005-2013 LAMP/EPFL
|
||||
* @author Martin Odersky
|
||||
*/
|
||||
|
||||
package org.apache.spark.repl
|
||||
|
||||
import scala.tools.nsc._
|
||||
import scala.tools.nsc.interpreter._
|
||||
|
||||
import scala.collection.{ mutable, immutable }
|
||||
import scala.PartialFunction.cond
|
||||
import scala.reflect.internal.Chars
|
||||
import scala.reflect.internal.Flags._
|
||||
import scala.language.implicitConversions
|
||||
|
||||
private[repl] trait SparkMemberHandlers {
|
||||
val intp: SparkIMain
|
||||
|
||||
import intp.{ Request, global, naming }
|
||||
import global._
|
||||
import naming._
|
||||
|
||||
private def codegenln(leadingPlus: Boolean, xs: String*): String = codegen(leadingPlus, (xs ++ Array("\n")): _*)
|
||||
private def codegenln(xs: String*): String = codegenln(true, xs: _*)
|
||||
|
||||
private def codegen(xs: String*): String = codegen(true, xs: _*)
|
||||
private def codegen(leadingPlus: Boolean, xs: String*): String = {
|
||||
val front = if (leadingPlus) "+ " else ""
|
||||
front + (xs map string2codeQuoted mkString " + ")
|
||||
}
|
||||
private implicit def name2string(name: Name) = name.toString
|
||||
|
||||
/** A traverser that finds all mentioned identifiers, i.e. things
|
||||
* that need to be imported. It might return extra names.
|
||||
*/
|
||||
private class ImportVarsTraverser extends Traverser {
|
||||
val importVars = new mutable.HashSet[Name]()
|
||||
|
||||
override def traverse(ast: Tree) = ast match {
|
||||
case Ident(name) =>
|
||||
// XXX this is obviously inadequate but it's going to require some effort
|
||||
// to get right.
|
||||
if (name.toString startsWith "x$") ()
|
||||
else importVars += name
|
||||
case _ => super.traverse(ast)
|
||||
}
|
||||
}
|
||||
private object ImportVarsTraverser {
|
||||
def apply(member: Tree) = {
|
||||
val ivt = new ImportVarsTraverser()
|
||||
ivt traverse member
|
||||
ivt.importVars.toList
|
||||
}
|
||||
}
|
||||
|
||||
def chooseHandler(member: Tree): MemberHandler = member match {
|
||||
case member: DefDef => new DefHandler(member)
|
||||
case member: ValDef => new ValHandler(member)
|
||||
case member: Assign => new AssignHandler(member)
|
||||
case member: ModuleDef => new ModuleHandler(member)
|
||||
case member: ClassDef => new ClassHandler(member)
|
||||
case member: TypeDef => new TypeAliasHandler(member)
|
||||
case member: Import => new ImportHandler(member)
|
||||
case DocDef(_, documented) => chooseHandler(documented)
|
||||
case member => new GenericHandler(member)
|
||||
}
|
||||
|
||||
sealed abstract class MemberDefHandler(override val member: MemberDef) extends MemberHandler(member) {
|
||||
def symbol = if (member.symbol eq null) NoSymbol else member.symbol
|
||||
def name: Name = member.name
|
||||
def mods: Modifiers = member.mods
|
||||
def keyword = member.keyword
|
||||
def prettyName = name.decode
|
||||
|
||||
override def definesImplicit = member.mods.isImplicit
|
||||
override def definesTerm: Option[TermName] = Some(name.toTermName) filter (_ => name.isTermName)
|
||||
override def definesType: Option[TypeName] = Some(name.toTypeName) filter (_ => name.isTypeName)
|
||||
override def definedSymbols = if (symbol eq NoSymbol) Nil else List(symbol)
|
||||
}
|
||||
|
||||
/** Class to handle one member among all the members included
|
||||
* in a single interpreter request.
|
||||
*/
|
||||
sealed abstract class MemberHandler(val member: Tree) {
|
||||
def definesImplicit = false
|
||||
def definesValue = false
|
||||
def isLegalTopLevel = false
|
||||
|
||||
def definesTerm = Option.empty[TermName]
|
||||
def definesType = Option.empty[TypeName]
|
||||
|
||||
lazy val referencedNames = ImportVarsTraverser(member)
|
||||
def importedNames = List[Name]()
|
||||
def definedNames = definesTerm.toList ++ definesType.toList
|
||||
def definedOrImported = definedNames ++ importedNames
|
||||
def definedSymbols = List[Symbol]()
|
||||
|
||||
def extraCodeToEvaluate(req: Request): String = ""
|
||||
def resultExtractionCode(req: Request): String = ""
|
||||
|
||||
private def shortName = this.getClass.toString split '.' last
|
||||
override def toString = shortName + referencedNames.mkString(" (refs: ", ", ", ")")
|
||||
}
|
||||
|
||||
class GenericHandler(member: Tree) extends MemberHandler(member)
|
||||
|
||||
class ValHandler(member: ValDef) extends MemberDefHandler(member) {
|
||||
val maxStringElements = 1000 // no need to mkString billions of elements
|
||||
override def definesValue = true
|
||||
|
||||
override def resultExtractionCode(req: Request): String = {
|
||||
val isInternal = isUserVarName(name) && req.lookupTypeOf(name) == "Unit"
|
||||
if (!mods.isPublic || isInternal) ""
|
||||
else {
|
||||
// if this is a lazy val we avoid evaluating it here
|
||||
val resultString =
|
||||
if (mods.isLazy) codegenln(false, "<lazy>")
|
||||
else any2stringOf(req fullPath name, maxStringElements)
|
||||
|
||||
val vidString =
|
||||
if (replProps.vids) """" + " @ " + "%%8x".format(System.identityHashCode(%s)) + " """.trim.format(req fullPath name)
|
||||
else ""
|
||||
|
||||
""" + "%s%s: %s = " + %s""".format(string2code(prettyName), vidString, string2code(req typeOf name), resultString)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class DefHandler(member: DefDef) extends MemberDefHandler(member) {
|
||||
private def vparamss = member.vparamss
|
||||
private def isMacro = member.symbol hasFlag MACRO
|
||||
// true if not a macro and 0-arity
|
||||
override def definesValue = !isMacro && flattensToEmpty(vparamss)
|
||||
override def resultExtractionCode(req: Request) =
|
||||
if (mods.isPublic) codegenln(name, ": ", req.typeOf(name)) else ""
|
||||
}
|
||||
|
||||
class AssignHandler(member: Assign) extends MemberHandler(member) {
|
||||
val Assign(lhs, rhs) = member
|
||||
val name = newTermName(freshInternalVarName())
|
||||
|
||||
override def definesTerm = Some(name)
|
||||
override def definesValue = true
|
||||
override def extraCodeToEvaluate(req: Request) =
|
||||
"""val %s = %s""".format(name, lhs)
|
||||
|
||||
/** Print out lhs instead of the generated varName */
|
||||
override def resultExtractionCode(req: Request) = {
|
||||
val lhsType = string2code(req lookupTypeOf name)
|
||||
val res = string2code(req fullPath name)
|
||||
""" + "%s: %s = " + %s + "\n" """.format(string2code(lhs.toString), lhsType, res) + "\n"
|
||||
}
|
||||
}
|
||||
|
||||
class ModuleHandler(module: ModuleDef) extends MemberDefHandler(module) {
|
||||
override def definesTerm = Some(name)
|
||||
override def definesValue = true
|
||||
override def isLegalTopLevel = true
|
||||
|
||||
override def resultExtractionCode(req: Request) = codegenln("defined module ", name)
|
||||
}
|
||||
|
||||
class ClassHandler(member: ClassDef) extends MemberDefHandler(member) {
|
||||
override def definesType = Some(name.toTypeName)
|
||||
override def definesTerm = Some(name.toTermName) filter (_ => mods.isCase)
|
||||
override def isLegalTopLevel = true
|
||||
|
||||
override def resultExtractionCode(req: Request) =
|
||||
codegenln("defined %s %s".format(keyword, name))
|
||||
}
|
||||
|
||||
class TypeAliasHandler(member: TypeDef) extends MemberDefHandler(member) {
|
||||
private def isAlias = mods.isPublic && treeInfo.isAliasTypeDef(member)
|
||||
override def definesType = Some(name.toTypeName) filter (_ => isAlias)
|
||||
|
||||
override def resultExtractionCode(req: Request) =
|
||||
codegenln("defined type alias ", name) + "\n"
|
||||
}
|
||||
|
||||
class ImportHandler(imp: Import) extends MemberHandler(imp) {
|
||||
val Import(expr, selectors) = imp
|
||||
def targetType: Type = intp.typeOfExpression("" + expr)
|
||||
override def isLegalTopLevel = true
|
||||
|
||||
def createImportForName(name: Name): String = {
|
||||
selectors foreach {
|
||||
case sel @ ImportSelector(old, _, `name`, _) => return "import %s.{ %s }".format(expr, sel)
|
||||
case _ => ()
|
||||
}
|
||||
"import %s.%s".format(expr, name)
|
||||
}
|
||||
// TODO: Need to track these specially to honor Predef masking attempts,
|
||||
// because they must be the leading imports in the code generated for each
|
||||
// line. We can use the same machinery as Contexts now, anyway.
|
||||
def isPredefImport = isReferenceToPredef(expr)
|
||||
|
||||
// wildcard imports, e.g. import foo._
|
||||
private def selectorWild = selectors filter (_.name == nme.USCOREkw)
|
||||
// renamed imports, e.g. import foo.{ bar => baz }
|
||||
private def selectorRenames = selectors map (_.rename) filterNot (_ == null)
|
||||
|
||||
/** Whether this import includes a wildcard import */
|
||||
val importsWildcard = selectorWild.nonEmpty
|
||||
|
||||
/** Whether anything imported is implicit .*/
|
||||
def importsImplicit = implicitSymbols.nonEmpty
|
||||
|
||||
def implicitSymbols = importedSymbols filter (_.isImplicit)
|
||||
def importedSymbols = individualSymbols ++ wildcardSymbols
|
||||
|
||||
lazy val individualSymbols: List[Symbol] =
|
||||
beforePickler(individualNames map (targetType nonPrivateMember _))
|
||||
|
||||
lazy val wildcardSymbols: List[Symbol] =
|
||||
if (importsWildcard) beforePickler(targetType.nonPrivateMembers.toList)
|
||||
else Nil
|
||||
|
||||
/** Complete list of names imported by a wildcard */
|
||||
lazy val wildcardNames: List[Name] = wildcardSymbols map (_.name)
|
||||
lazy val individualNames: List[Name] = selectorRenames filterNot (_ == nme.USCOREkw) flatMap (_.bothNames)
|
||||
|
||||
/** The names imported by this statement */
|
||||
override lazy val importedNames: List[Name] = wildcardNames ++ individualNames
|
||||
lazy val importsSymbolNamed: Set[String] = importedNames map (_.toString) toSet
|
||||
|
||||
def importString = imp.toString
|
||||
override def resultExtractionCode(req: Request) = codegenln(importString) + "\n"
|
||||
}
|
||||
}
|
|
@ -1,31 +0,0 @@
|
|||
/*
|
||||
* 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.repl
|
||||
|
||||
import scala.tools.nsc.Settings
|
||||
|
||||
/**
|
||||
* <i>scala.tools.nsc.Settings</i> implementation adding Spark-specific REPL
|
||||
* command line options.
|
||||
*/
|
||||
private[repl] class SparkRunnerSettings(error: String => Unit) extends Settings(error) {
|
||||
val loadfiles = MultiStringSetting(
|
||||
"-i",
|
||||
"file",
|
||||
"load a file (assumes the code is given interactively)")
|
||||
}
|
|
@ -1,366 +0,0 @@
|
|||
/*
|
||||
* 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.repl
|
||||
|
||||
import java.io._
|
||||
import java.net.URLClassLoader
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
import org.apache.spark.{SparkContext, SparkFunSuite}
|
||||
import org.apache.commons.lang3.StringEscapeUtils
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
|
||||
class ReplSuite extends SparkFunSuite {
|
||||
|
||||
def runInterpreter(master: String, input: String): String = {
|
||||
val CONF_EXECUTOR_CLASSPATH = "spark.executor.extraClassPath"
|
||||
|
||||
val in = new BufferedReader(new StringReader(input + "\n"))
|
||||
val out = new StringWriter()
|
||||
val cl = getClass.getClassLoader
|
||||
var paths = new ArrayBuffer[String]
|
||||
if (cl.isInstanceOf[URLClassLoader]) {
|
||||
val urlLoader = cl.asInstanceOf[URLClassLoader]
|
||||
for (url <- urlLoader.getURLs) {
|
||||
if (url.getProtocol == "file") {
|
||||
paths += url.getFile
|
||||
}
|
||||
}
|
||||
}
|
||||
val classpath = paths.map(new File(_).getAbsolutePath).mkString(File.pathSeparator)
|
||||
|
||||
val oldExecutorClasspath = System.getProperty(CONF_EXECUTOR_CLASSPATH)
|
||||
System.setProperty(CONF_EXECUTOR_CLASSPATH, classpath)
|
||||
|
||||
val interp = new SparkILoop(in, new PrintWriter(out), master)
|
||||
org.apache.spark.repl.Main.interp = interp
|
||||
interp.process(Array("-classpath", classpath))
|
||||
org.apache.spark.repl.Main.interp = null
|
||||
if (interp.sparkContext != null) {
|
||||
interp.sparkContext.stop()
|
||||
}
|
||||
if (oldExecutorClasspath != null) {
|
||||
System.setProperty(CONF_EXECUTOR_CLASSPATH, oldExecutorClasspath)
|
||||
} else {
|
||||
System.clearProperty(CONF_EXECUTOR_CLASSPATH)
|
||||
}
|
||||
return out.toString
|
||||
}
|
||||
|
||||
def assertContains(message: String, output: String) {
|
||||
val isContain = output.contains(message)
|
||||
assert(isContain,
|
||||
"Interpreter output did not contain '" + message + "':\n" + output)
|
||||
}
|
||||
|
||||
def assertDoesNotContain(message: String, output: String) {
|
||||
val isContain = output.contains(message)
|
||||
assert(!isContain,
|
||||
"Interpreter output contained '" + message + "':\n" + output)
|
||||
}
|
||||
|
||||
test("propagation of local properties") {
|
||||
// A mock ILoop that doesn't install the SIGINT handler.
|
||||
class ILoop(out: PrintWriter) extends SparkILoop(None, out, None) {
|
||||
settings = new scala.tools.nsc.Settings
|
||||
settings.usejavacp.value = true
|
||||
org.apache.spark.repl.Main.interp = this
|
||||
override def createInterpreter() {
|
||||
intp = new SparkILoopInterpreter
|
||||
intp.setContextClassLoader()
|
||||
}
|
||||
}
|
||||
|
||||
val out = new StringWriter()
|
||||
val interp = new ILoop(new PrintWriter(out))
|
||||
interp.sparkContext = new SparkContext("local", "repl-test")
|
||||
interp.createInterpreter()
|
||||
interp.intp.initialize()
|
||||
interp.sparkContext.setLocalProperty("someKey", "someValue")
|
||||
|
||||
// Make sure the value we set in the caller to interpret is propagated in the thread that
|
||||
// interprets the command.
|
||||
interp.interpret("org.apache.spark.repl.Main.interp.sparkContext.getLocalProperty(\"someKey\")")
|
||||
assert(out.toString.contains("someValue"))
|
||||
|
||||
interp.sparkContext.stop()
|
||||
System.clearProperty("spark.driver.port")
|
||||
}
|
||||
|
||||
test("simple foreach with accumulator") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|val accum = sc.longAccumulator
|
||||
|sc.parallelize(1 to 10).foreach(x => accum.add(x))
|
||||
|accum.value
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res1: Long = 55", output)
|
||||
}
|
||||
|
||||
test("external vars") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|var v = 7
|
||||
|sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_)
|
||||
|v = 10
|
||||
|sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_)
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Int = 70", output)
|
||||
assertContains("res1: Int = 100", output)
|
||||
}
|
||||
|
||||
test("external classes") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|class C {
|
||||
|def foo = 5
|
||||
|}
|
||||
|sc.parallelize(1 to 10).map(x => (new C).foo).collect().reduceLeft(_+_)
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Int = 50", output)
|
||||
}
|
||||
|
||||
test("external functions") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|def double(x: Int) = x + x
|
||||
|sc.parallelize(1 to 10).map(x => double(x)).collect().reduceLeft(_+_)
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Int = 110", output)
|
||||
}
|
||||
|
||||
test("external functions that access vars") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|var v = 7
|
||||
|def getV() = v
|
||||
|sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_)
|
||||
|v = 10
|
||||
|sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_)
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Int = 70", output)
|
||||
assertContains("res1: Int = 100", output)
|
||||
}
|
||||
|
||||
test("broadcast vars") {
|
||||
// Test that the value that a broadcast var had when it was created is used,
|
||||
// even if that variable is then modified in the driver program
|
||||
// TODO: This doesn't actually work for arrays when we run in local mode!
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|var array = new Array[Int](5)
|
||||
|val broadcastArray = sc.broadcast(array)
|
||||
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect()
|
||||
|array(0) = 5
|
||||
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output)
|
||||
assertContains("res2: Array[Int] = Array(5, 0, 0, 0, 0)", output)
|
||||
}
|
||||
|
||||
test("interacting with files") {
|
||||
val tempDir = Utils.createTempDir()
|
||||
val out = new FileWriter(tempDir + "/input")
|
||||
out.write("Hello world!\n")
|
||||
out.write("What's up?\n")
|
||||
out.write("Goodbye\n")
|
||||
out.close()
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|var file = sc.textFile("%s").cache()
|
||||
|file.count()
|
||||
|file.count()
|
||||
|file.count()
|
||||
""".stripMargin.format(StringEscapeUtils.escapeJava(
|
||||
tempDir.getAbsolutePath + File.separator + "input")))
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Long = 3", output)
|
||||
assertContains("res1: Long = 3", output)
|
||||
assertContains("res2: Long = 3", output)
|
||||
Utils.deleteRecursively(tempDir)
|
||||
}
|
||||
|
||||
test("local-cluster mode") {
|
||||
val output = runInterpreter("local-cluster[1,1,1024]",
|
||||
"""
|
||||
|var v = 7
|
||||
|def getV() = v
|
||||
|sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_)
|
||||
|v = 10
|
||||
|sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_)
|
||||
|var array = new Array[Int](5)
|
||||
|val broadcastArray = sc.broadcast(array)
|
||||
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect()
|
||||
|array(0) = 5
|
||||
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Int = 70", output)
|
||||
assertContains("res1: Int = 100", output)
|
||||
assertContains("res2: Array[Int] = Array(0, 0, 0, 0, 0)", output)
|
||||
assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output)
|
||||
}
|
||||
|
||||
test("SPARK-1199 two instances of same class don't type check.") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|case class Sum(exp: String, exp2: String)
|
||||
|val a = Sum("A", "B")
|
||||
|def b(a: Sum): String = a match { case Sum(_, _) => "Found Sum" }
|
||||
|b(a)
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
}
|
||||
|
||||
test("SPARK-2452 compound statements.") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|val x = 4 ; def f() = x
|
||||
|f()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
}
|
||||
|
||||
test("SPARK-2576 importing SQLContext.implicits._") {
|
||||
// We need to use local-cluster to test this case.
|
||||
val output = runInterpreter("local-cluster[1,1,1024]",
|
||||
"""
|
||||
|val sqlContext = new org.apache.spark.sql.SQLContext(sc)
|
||||
|import sqlContext.implicits._
|
||||
|case class TestCaseClass(value: Int)
|
||||
|sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF().collect()
|
||||
|
|
||||
|// Test Dataset Serialization in the REPL
|
||||
|Seq(TestCaseClass(1)).toDS().collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
}
|
||||
|
||||
test("SPARK-8461 SQL with codegen") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|val sqlContext = new org.apache.spark.sql.SQLContext(sc)
|
||||
|sqlContext.setConf("spark.sql.codegen", "true")
|
||||
|sqlContext.range(0, 100).filter('id > 50).count()
|
||||
""".stripMargin)
|
||||
assertContains("Long = 49", output)
|
||||
assertDoesNotContain("java.lang.ClassNotFoundException", output)
|
||||
}
|
||||
|
||||
test("Datasets and encoders") {
|
||||
val output = runInterpreter("local",
|
||||
"""
|
||||
|import org.apache.spark.sql.functions._
|
||||
|import org.apache.spark.sql.{Encoder, Encoders}
|
||||
|import org.apache.spark.sql.expressions.Aggregator
|
||||
|import org.apache.spark.sql.TypedColumn
|
||||
|val simpleSum = new Aggregator[Int, Int, Int] {
|
||||
| def zero: Int = 0 // The initial value.
|
||||
| def reduce(b: Int, a: Int) = b + a // Add an element to the running total
|
||||
| def merge(b1: Int, b2: Int) = b1 + b2 // Merge intermediate values.
|
||||
| def finish(b: Int) = b // Return the final result.
|
||||
| def bufferEncoder: Encoder[Int] = Encoders.scalaInt
|
||||
| def outputEncoder: Encoder[Int] = Encoders.scalaInt
|
||||
|}.toColumn
|
||||
|
|
||||
|val ds = Seq(1, 2, 3, 4).toDS()
|
||||
|ds.select(simpleSum).collect
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
}
|
||||
|
||||
test("SPARK-2632 importing a method from non serializable class and not using it.") {
|
||||
val output = runInterpreter("local-cluster[1,1,1024]",
|
||||
"""
|
||||
|class TestClass() { def testMethod = 3 }
|
||||
|val t = new TestClass
|
||||
|import t.testMethod
|
||||
|case class TestCaseClass(value: Int)
|
||||
|sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
}
|
||||
|
||||
if (System.getenv("MESOS_NATIVE_JAVA_LIBRARY") != null) {
|
||||
test("running on Mesos") {
|
||||
val output = runInterpreter("localquiet",
|
||||
"""
|
||||
|var v = 7
|
||||
|def getV() = v
|
||||
|sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_)
|
||||
|v = 10
|
||||
|sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_)
|
||||
|var array = new Array[Int](5)
|
||||
|val broadcastArray = sc.broadcast(array)
|
||||
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect()
|
||||
|array(0) = 5
|
||||
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("res0: Int = 70", output)
|
||||
assertContains("res1: Int = 100", output)
|
||||
assertContains("res2: Array[Int] = Array(0, 0, 0, 0, 0)", output)
|
||||
assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output)
|
||||
}
|
||||
}
|
||||
|
||||
test("collecting objects of class defined in repl") {
|
||||
val output = runInterpreter("local[2]",
|
||||
"""
|
||||
|case class Foo(i: Int)
|
||||
|val ret = sc.parallelize((1 to 100).map(Foo), 10).collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("ret: Array[Foo] = Array(Foo(1),", output)
|
||||
}
|
||||
|
||||
test("collecting objects of class defined in repl - shuffling") {
|
||||
val output = runInterpreter("local-cluster[1,1,1024]",
|
||||
"""
|
||||
|case class Foo(i: Int)
|
||||
|val list = List((1, Foo(1)), (1, Foo(2)))
|
||||
|val ret = sc.parallelize(list).groupByKey().collect()
|
||||
""".stripMargin)
|
||||
assertDoesNotContain("error:", output)
|
||||
assertDoesNotContain("Exception", output)
|
||||
assertContains("ret: Array[(Int, Iterable[Foo])] = Array((1,", output)
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.spark.sql.execution;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import scala.collection.AbstractIterator;
|
||||
import scala.collection.Iterator;
|
||||
import scala.math.Ordering;
|
||||
|
||||
|
@ -28,7 +29,6 @@ import org.apache.spark.SparkEnv;
|
|||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
|
||||
import org.apache.spark.sql.catalyst.util.AbstractScalaRowIterator;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.unsafe.Platform;
|
||||
import org.apache.spark.util.collection.unsafe.sort.PrefixComparator;
|
||||
|
@ -145,7 +145,7 @@ public final class UnsafeExternalRowSorter {
|
|||
// here in order to prevent memory leaks.
|
||||
cleanupResources();
|
||||
}
|
||||
return new AbstractScalaRowIterator<UnsafeRow>() {
|
||||
return new AbstractIterator<UnsafeRow>() {
|
||||
|
||||
private final int numFields = schema.length();
|
||||
private UnsafeRow row = new UnsafeRow(numFields);
|
||||
|
|
|
@ -41,8 +41,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
// Since we are creating a runtime mirror using the class loader of current thread,
|
||||
// we need to use def at here. So, every time we call mirror, it is using the
|
||||
// class loader of the current thread.
|
||||
// SPARK-13640: Synchronize this because universe.runtimeMirror is not thread-safe in Scala 2.10.
|
||||
override def mirror: universe.Mirror = ScalaReflectionLock.synchronized {
|
||||
override def mirror: universe.Mirror = {
|
||||
universe.runtimeMirror(Thread.currentThread().getContextClassLoader)
|
||||
}
|
||||
|
||||
|
@ -62,7 +61,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
*/
|
||||
def dataTypeFor[T : TypeTag]: DataType = dataTypeFor(localTypeOf[T])
|
||||
|
||||
private def dataTypeFor(tpe: `Type`): DataType = ScalaReflectionLock.synchronized {
|
||||
private def dataTypeFor(tpe: `Type`): DataType = {
|
||||
tpe match {
|
||||
case t if t <:< definitions.IntTpe => IntegerType
|
||||
case t if t <:< definitions.LongTpe => LongType
|
||||
|
@ -94,7 +93,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
* Special handling is performed for primitive types to map them back to their raw
|
||||
* JVM form instead of the Scala Array that handles auto boxing.
|
||||
*/
|
||||
private def arrayClassFor(tpe: `Type`): ObjectType = ScalaReflectionLock.synchronized {
|
||||
private def arrayClassFor(tpe: `Type`): ObjectType = {
|
||||
val cls = tpe match {
|
||||
case t if t <:< definitions.IntTpe => classOf[Array[Int]]
|
||||
case t if t <:< definitions.LongTpe => classOf[Array[Long]]
|
||||
|
@ -141,7 +140,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
private def deserializerFor(
|
||||
tpe: `Type`,
|
||||
path: Option[Expression],
|
||||
walkedTypePath: Seq[String]): Expression = ScalaReflectionLock.synchronized {
|
||||
walkedTypePath: Seq[String]): Expression = {
|
||||
|
||||
/** Returns the current path with a sub-field extracted. */
|
||||
def addToPath(part: String, dataType: DataType, walkedTypePath: Seq[String]): Expression = {
|
||||
|
@ -329,8 +328,8 @@ object ScalaReflection extends ScalaReflection {
|
|||
}
|
||||
}
|
||||
|
||||
val companion = t.normalize.typeSymbol.companionSymbol.typeSignature
|
||||
val cls = companion.member(newTermName("newBuilder")) match {
|
||||
val companion = t.dealias.typeSymbol.companion.typeSignature
|
||||
val cls = companion.member(TermName("newBuilder")) match {
|
||||
case NoSymbol if t <:< localTypeOf[Seq[_]] => classOf[Seq[_]]
|
||||
case NoSymbol if t <:< localTypeOf[scala.collection.Set[_]] =>
|
||||
classOf[scala.collection.Set[_]]
|
||||
|
@ -349,7 +348,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
mirror.runtimeClass(t.typeSymbol.asClass)
|
||||
)
|
||||
|
||||
case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) =>
|
||||
case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) =>
|
||||
val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance()
|
||||
val obj = NewInstance(
|
||||
udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(),
|
||||
|
@ -436,7 +435,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
inputObject: Expression,
|
||||
tpe: `Type`,
|
||||
walkedTypePath: Seq[String],
|
||||
seenTypeSet: Set[`Type`] = Set.empty): Expression = ScalaReflectionLock.synchronized {
|
||||
seenTypeSet: Set[`Type`] = Set.empty): Expression = {
|
||||
|
||||
def toCatalystArray(input: Expression, elementType: `Type`): Expression = {
|
||||
dataTypeFor(elementType) match {
|
||||
|
@ -591,7 +590,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
case t if t <:< localTypeOf[java.lang.Boolean] =>
|
||||
Invoke(inputObject, "booleanValue", BooleanType)
|
||||
|
||||
case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) =>
|
||||
case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) =>
|
||||
val udt = getClassFromType(t)
|
||||
.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance()
|
||||
val obj = NewInstance(
|
||||
|
@ -643,7 +642,7 @@ object ScalaReflection extends ScalaReflection {
|
|||
* Returns true if the given type is option of product type, e.g. `Option[Tuple2]`. Note that,
|
||||
* we also treat [[DefinedByConstructorParams]] as product type.
|
||||
*/
|
||||
def optionOfProductType(tpe: `Type`): Boolean = ScalaReflectionLock.synchronized {
|
||||
def optionOfProductType(tpe: `Type`): Boolean = {
|
||||
tpe match {
|
||||
case t if t <:< localTypeOf[Option[_]] =>
|
||||
val TypeRef(_, _, Seq(optType)) = t
|
||||
|
@ -705,9 +704,9 @@ object ScalaReflection extends ScalaReflection {
|
|||
def schemaFor[T: TypeTag]: Schema = schemaFor(localTypeOf[T])
|
||||
|
||||
/** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */
|
||||
def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized {
|
||||
def schemaFor(tpe: `Type`): Schema = {
|
||||
tpe match {
|
||||
case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) =>
|
||||
case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) =>
|
||||
val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance()
|
||||
Schema(udt, nullable = true)
|
||||
case t if UDTRegistration.exists(getClassNameFromType(t)) =>
|
||||
|
@ -814,10 +813,9 @@ trait ScalaReflection {
|
|||
*
|
||||
* @see SPARK-5281
|
||||
*/
|
||||
// SPARK-13640: Synchronize this because TypeTag.tpe is not thread-safe in Scala 2.10.
|
||||
def localTypeOf[T: TypeTag]: `Type` = ScalaReflectionLock.synchronized {
|
||||
def localTypeOf[T: TypeTag]: `Type` = {
|
||||
val tag = implicitly[TypeTag[T]]
|
||||
tag.in(mirror).tpe.normalize
|
||||
tag.in(mirror).tpe.dealias
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -866,9 +864,9 @@ trait ScalaReflection {
|
|||
}
|
||||
|
||||
protected def constructParams(tpe: Type): Seq[Symbol] = {
|
||||
val constructorSymbol = tpe.member(nme.CONSTRUCTOR)
|
||||
val constructorSymbol = tpe.member(termNames.CONSTRUCTOR)
|
||||
val params = if (constructorSymbol.isMethod) {
|
||||
constructorSymbol.asMethod.paramss
|
||||
constructorSymbol.asMethod.paramLists
|
||||
} else {
|
||||
// Find the primary constructor, and use its parameter ordering.
|
||||
val primaryConstructorSymbol: Option[Symbol] = constructorSymbol.asTerm.alternatives.find(
|
||||
|
@ -876,7 +874,7 @@ trait ScalaReflection {
|
|||
if (primaryConstructorSymbol.isEmpty) {
|
||||
sys.error("Internal SQL error: Product object did not have a primary constructor.")
|
||||
} else {
|
||||
primaryConstructorSymbol.get.asMethod.paramss
|
||||
primaryConstructorSymbol.get.asMethod.paramLists
|
||||
}
|
||||
}
|
||||
params.flatten
|
||||
|
|
|
@ -1518,9 +1518,9 @@ class Analyzer(
|
|||
*/
|
||||
def unapply(e: Expression): Option[(Generator, Seq[String], Boolean)] = e match {
|
||||
case Alias(GeneratorOuter(g: Generator), name) if g.resolved => Some((g, name :: Nil, true))
|
||||
case MultiAlias(GeneratorOuter(g: Generator), names) if g.resolved => Some(g, names, true)
|
||||
case MultiAlias(GeneratorOuter(g: Generator), names) if g.resolved => Some((g, names, true))
|
||||
case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil, false))
|
||||
case MultiAlias(g: Generator, names) if g.resolved => Some(g, names, false)
|
||||
case MultiAlias(g: Generator, names) if g.resolved => Some((g, names, false))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
|
||||
package org.apache.spark.sql.catalyst.expressions
|
||||
|
||||
import scala.reflect.internal.util.AbstractFileClassLoader
|
||||
|
||||
import org.apache.spark.sql.catalyst.rules
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
|
@ -51,7 +53,7 @@ package object codegen {
|
|||
val classLoader =
|
||||
generatedClass
|
||||
.getClassLoader
|
||||
.asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader]
|
||||
.asInstanceOf[AbstractFileClassLoader]
|
||||
val generatedBytes = classLoader.classBytes(generatedClass.getName)
|
||||
|
||||
val packageDir = new java.io.File(dumpDirectory, generatedClass.getPackage.getName)
|
||||
|
|
|
@ -30,7 +30,7 @@ object NamedExpression {
|
|||
private val curId = new java.util.concurrent.atomic.AtomicLong()
|
||||
private[expressions] val jvmId = UUID.randomUUID()
|
||||
def newExprId: ExprId = ExprId(curId.getAndIncrement(), jvmId)
|
||||
def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType)
|
||||
def unapply(expr: NamedExpression): Option[(String, DataType)] = Some((expr.name, expr.dataType))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,10 +22,4 @@ package org.apache.spark.sql
|
|||
* considered an internal API to Spark SQL and are subject to change between minor releases.
|
||||
*/
|
||||
package object catalyst {
|
||||
/**
|
||||
* A JVM-global lock that should be used to prevent thread safety issues when using things in
|
||||
* scala.reflect.*. Note that Scala Reflection API is made thread-safe in 2.11, but not yet for
|
||||
* 2.10.* builds. See SI-6240 for more details.
|
||||
*/
|
||||
protected[sql] object ScalaReflectionLock
|
||||
}
|
||||
|
|
|
@ -1,30 +0,0 @@
|
|||
/*
|
||||
* 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.catalyst.util
|
||||
|
||||
/**
|
||||
* Shim to allow us to implement [[scala.Iterator]] in Java. Scala 2.11+ has an AbstractIterator
|
||||
* class for this, but that class is `private[scala]` in 2.10. We need to explicitly fix this to
|
||||
* `Row` in order to work around a spurious IntelliJ compiler error. This cannot be an abstract
|
||||
* class because that leads to compilation errors under Scala 2.11.
|
||||
*/
|
||||
class AbstractScalaRowIterator[T] extends Iterator[T] {
|
||||
override def hasNext: Boolean = throw new NotImplementedError
|
||||
|
||||
override def next(): T = throw new NotImplementedError
|
||||
}
|
|
@ -21,7 +21,6 @@ import scala.math.Ordering
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
import org.apache.spark.sql.catalyst.util.TypeUtils
|
||||
|
||||
|
||||
|
@ -37,7 +36,7 @@ class BinaryType private() extends AtomicType {
|
|||
|
||||
private[sql] type InternalType = Array[Byte]
|
||||
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
|
||||
private[sql] val ordering = new Ordering[InternalType] {
|
||||
def compare(x: Array[Byte], y: Array[Byte]): Int = {
|
||||
|
|
|
@ -21,7 +21,6 @@ import scala.math.Ordering
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
|
||||
/**
|
||||
|
@ -35,7 +34,7 @@ class BooleanType private() extends AtomicType {
|
|||
// this type. Otherwise, the companion object would be of type "BooleanType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Boolean
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,7 +21,6 @@ import scala.math.{Integral, Numeric, Ordering}
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
/**
|
||||
* The data type representing `Byte` values. Please use the singleton `DataTypes.ByteType`.
|
||||
|
@ -34,7 +33,7 @@ class ByteType private() extends IntegralType {
|
|||
// this type. Otherwise, the companion object would be of type "ByteType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Byte
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = implicitly[Numeric[Byte]]
|
||||
private[sql] val integral = implicitly[Integral[Byte]]
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
|
|
@ -21,8 +21,6 @@ import scala.math.Ordering
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
|
||||
/**
|
||||
* A date type, supporting "0001-01-01" through "9999-12-31".
|
||||
|
@ -40,7 +38,7 @@ class DateType private() extends AtomicType {
|
|||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Int
|
||||
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
||||
|
|
|
@ -23,7 +23,6 @@ import scala.reflect.runtime.universe.typeTag
|
|||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.AnalysisException
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
|
||||
|
||||
|
@ -57,7 +56,7 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType {
|
|||
def this() = this(10)
|
||||
|
||||
private[sql] type InternalType = Decimal
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = Decimal.DecimalIsFractional
|
||||
private[sql] val fractional = Decimal.DecimalIsFractional
|
||||
private[sql] val ordering = Decimal.DecimalIsFractional
|
||||
|
|
|
@ -22,7 +22,6 @@ import scala.math.Numeric.DoubleAsIfIntegral
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
/**
|
||||
|
@ -36,7 +35,7 @@ class DoubleType private() extends FractionalType {
|
|||
// this type. Otherwise, the companion object would be of type "DoubleType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Double
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = implicitly[Numeric[Double]]
|
||||
private[sql] val fractional = implicitly[Fractional[Double]]
|
||||
private[sql] val ordering = new Ordering[Double] {
|
||||
|
|
|
@ -22,7 +22,6 @@ import scala.math.Numeric.FloatAsIfIntegral
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
/**
|
||||
|
@ -36,7 +35,7 @@ class FloatType private() extends FractionalType {
|
|||
// this type. Otherwise, the companion object would be of type "FloatType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Float
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = implicitly[Numeric[Float]]
|
||||
private[sql] val fractional = implicitly[Fractional[Float]]
|
||||
private[sql] val ordering = new Ordering[Float] {
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.spark.sql.types
|
|||
import scala.math.Ordering
|
||||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
/**
|
||||
|
@ -32,9 +31,7 @@ sealed abstract class HiveStringType extends AtomicType {
|
|||
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized {
|
||||
typeTag[InternalType]
|
||||
}
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
|
||||
override def defaultSize: Int = length
|
||||
|
||||
|
|
|
@ -21,8 +21,6 @@ import scala.math.{Integral, Numeric, Ordering}
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
|
||||
/**
|
||||
* The data type representing `Int` values. Please use the singleton `DataTypes.IntegerType`.
|
||||
|
@ -35,7 +33,7 @@ class IntegerType private() extends IntegralType {
|
|||
// this type. Otherwise, the companion object would be of type "IntegerType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Int
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = implicitly[Numeric[Int]]
|
||||
private[sql] val integral = implicitly[Integral[Int]]
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
|
|
@ -21,7 +21,6 @@ import scala.math.{Integral, Numeric, Ordering}
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
/**
|
||||
* The data type representing `Long` values. Please use the singleton `DataTypes.LongType`.
|
||||
|
@ -34,7 +33,7 @@ class LongType private() extends IntegralType {
|
|||
// this type. Otherwise, the companion object would be of type "LongType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Long
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = implicitly[Numeric[Long]]
|
||||
private[sql] val integral = implicitly[Integral[Long]]
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
|
|
@ -21,7 +21,6 @@ import scala.math.{Integral, Numeric, Ordering}
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
/**
|
||||
* The data type representing `Short` values. Please use the singleton `DataTypes.ShortType`.
|
||||
|
@ -34,7 +33,7 @@ class ShortType private() extends IntegralType {
|
|||
// this type. Otherwise, the companion object would be of type "ShortType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Short
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val numeric = implicitly[Numeric[Short]]
|
||||
private[sql] val integral = implicitly[Integral[Short]]
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
|
|
@ -21,7 +21,6 @@ import scala.math.Ordering
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
/**
|
||||
|
@ -35,7 +34,7 @@ class StringType private() extends AtomicType {
|
|||
// this type. Otherwise, the companion object would be of type "StringType$" in byte code.
|
||||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = UTF8String
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,8 +21,6 @@ import scala.math.Ordering
|
|||
import scala.reflect.runtime.universe.typeTag
|
||||
|
||||
import org.apache.spark.annotation.InterfaceStability
|
||||
import org.apache.spark.sql.catalyst.ScalaReflectionLock
|
||||
|
||||
|
||||
/**
|
||||
* The data type representing `java.sql.Timestamp` values.
|
||||
|
@ -37,7 +35,7 @@ class TimestampType private() extends AtomicType {
|
|||
// Defined with a private constructor so the companion object is the only possible instantiation.
|
||||
private[sql] type InternalType = Long
|
||||
|
||||
@transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] }
|
||||
@transient private[sql] lazy val tag = typeTag[InternalType]
|
||||
|
||||
private[sql] val ordering = implicitly[Ordering[InternalType]]
|
||||
|
||||
|
|
|
@ -70,27 +70,27 @@ class JoinOptimizationSuite extends PlanTest {
|
|||
|
||||
testExtract(x, None)
|
||||
testExtract(x.where("x.b".attr === 1), None)
|
||||
testExtract(x.join(y), Some(Seq(x, y), Seq()))
|
||||
testExtract(x.join(y), Some((Seq(x, y), Seq())))
|
||||
testExtract(x.join(y, condition = Some("x.b".attr === "y.d".attr)),
|
||||
Some(Seq(x, y), Seq("x.b".attr === "y.d".attr)))
|
||||
Some((Seq(x, y), Seq("x.b".attr === "y.d".attr))))
|
||||
testExtract(x.join(y).where("x.b".attr === "y.d".attr),
|
||||
Some(Seq(x, y), Seq("x.b".attr === "y.d".attr)))
|
||||
testExtract(x.join(y).join(z), Some(Seq(x, y, z), Seq()))
|
||||
Some((Seq(x, y), Seq("x.b".attr === "y.d".attr))))
|
||||
testExtract(x.join(y).join(z), Some((Seq(x, y, z), Seq())))
|
||||
testExtract(x.join(y).where("x.b".attr === "y.d".attr).join(z),
|
||||
Some(Seq(x, y, z), Seq("x.b".attr === "y.d".attr)))
|
||||
testExtract(x.join(y).join(x.join(z)), Some(Seq(x, y, x.join(z)), Seq()))
|
||||
Some((Seq(x, y, z), Seq("x.b".attr === "y.d".attr))))
|
||||
testExtract(x.join(y).join(x.join(z)), Some((Seq(x, y, x.join(z)), Seq())))
|
||||
testExtract(x.join(y).join(x.join(z)).where("x.b".attr === "y.d".attr),
|
||||
Some(Seq(x, y, x.join(z)), Seq("x.b".attr === "y.d".attr)))
|
||||
Some((Seq(x, y, x.join(z)), Seq("x.b".attr === "y.d".attr))))
|
||||
|
||||
testExtractCheckCross(x.join(y, Cross), Some(Seq((x, Cross), (y, Cross)), Seq()))
|
||||
testExtractCheckCross(x.join(y, Cross), Some((Seq((x, Cross), (y, Cross)), Seq())))
|
||||
testExtractCheckCross(x.join(y, Cross).join(z, Cross),
|
||||
Some(Seq((x, Cross), (y, Cross), (z, Cross)), Seq()))
|
||||
Some((Seq((x, Cross), (y, Cross), (z, Cross)), Seq())))
|
||||
testExtractCheckCross(x.join(y, Cross, Some("x.b".attr === "y.d".attr)).join(z, Cross),
|
||||
Some(Seq((x, Cross), (y, Cross), (z, Cross)), Seq("x.b".attr === "y.d".attr)))
|
||||
Some((Seq((x, Cross), (y, Cross), (z, Cross)), Seq("x.b".attr === "y.d".attr))))
|
||||
testExtractCheckCross(x.join(y, Inner, Some("x.b".attr === "y.d".attr)).join(z, Cross),
|
||||
Some(Seq((x, Inner), (y, Inner), (z, Cross)), Seq("x.b".attr === "y.d".attr)))
|
||||
Some((Seq((x, Inner), (y, Inner), (z, Cross)), Seq("x.b".attr === "y.d".attr))))
|
||||
testExtractCheckCross(x.join(y, Cross, Some("x.b".attr === "y.d".attr)).join(z, Inner),
|
||||
Some(Seq((x, Cross), (y, Cross), (z, Inner)), Seq("x.b".attr === "y.d".attr)))
|
||||
Some((Seq((x, Cross), (y, Cross), (z, Inner)), Seq("x.b".attr === "y.d".attr))))
|
||||
}
|
||||
|
||||
test("reorder inner joins") {
|
||||
|
|
|
@ -43,8 +43,8 @@ class OptimizeCodegenSuite extends PlanTest {
|
|||
CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)).toCodegen())
|
||||
|
||||
assertEquivalent(
|
||||
CaseWhen(List.fill(100)(TrueLiteral, Literal(1)), Literal(2)),
|
||||
CaseWhen(List.fill(100)(TrueLiteral, Literal(1)), Literal(2)))
|
||||
CaseWhen(List.fill(100)((TrueLiteral, Literal(1))), Literal(2)),
|
||||
CaseWhen(List.fill(100)((TrueLiteral, Literal(1))), Literal(2)))
|
||||
}
|
||||
|
||||
test("Nested CaseWhen Codegen.") {
|
||||
|
|
|
@ -505,7 +505,7 @@ class Column(val expr: Expression) extends Logging {
|
|||
*/
|
||||
def when(condition: Column, value: Any): Column = this.expr match {
|
||||
case CaseWhen(branches, None) =>
|
||||
withExpr { CaseWhen(branches :+ (condition.expr, lit(value).expr)) }
|
||||
withExpr { CaseWhen(branches :+ ((condition.expr, lit(value).expr))) }
|
||||
case CaseWhen(branches, Some(_)) =>
|
||||
throw new IllegalArgumentException(
|
||||
"when() cannot be applied once otherwise() is applied")
|
||||
|
|
|
@ -133,20 +133,20 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic
|
|||
case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _)
|
||||
if fsRelation.partitionSchema.nonEmpty =>
|
||||
val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l)
|
||||
Some(AttributeSet(partAttrs), l)
|
||||
Some((AttributeSet(partAttrs), l))
|
||||
|
||||
case relation: CatalogRelation if relation.tableMeta.partitionColumnNames.nonEmpty =>
|
||||
val partAttrs = getPartitionAttrs(relation.tableMeta.partitionColumnNames, relation)
|
||||
Some(AttributeSet(partAttrs), relation)
|
||||
Some((AttributeSet(partAttrs), relation))
|
||||
|
||||
case p @ Project(projectList, child) if projectList.forall(_.deterministic) =>
|
||||
unapply(child).flatMap { case (partAttrs, relation) =>
|
||||
if (p.references.subsetOf(partAttrs)) Some(p.outputSet, relation) else None
|
||||
if (p.references.subsetOf(partAttrs)) Some((p.outputSet, relation)) else None
|
||||
}
|
||||
|
||||
case f @ Filter(condition, child) if condition.deterministic =>
|
||||
unapply(child).flatMap { case (partAttrs, relation) =>
|
||||
if (f.references.subsetOf(partAttrs)) Some(partAttrs, relation) else None
|
||||
if (f.references.subsetOf(partAttrs)) Some((partAttrs, relation)) else None
|
||||
}
|
||||
|
||||
case _ => None
|
||||
|
|
|
@ -155,7 +155,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
|
|||
def toHiveStructString(a: (Any, DataType)): String = a match {
|
||||
case (struct: Row, StructType(fields)) =>
|
||||
struct.toSeq.zip(fields).map {
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}"""
|
||||
}.mkString("{", ",", "}")
|
||||
case (seq: Seq[_], ArrayType(typ, _)) =>
|
||||
seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
|
||||
|
@ -173,7 +173,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
|
|||
a match {
|
||||
case (struct: Row, StructType(fields)) =>
|
||||
struct.toSeq.zip(fields).map {
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}"""
|
||||
}.mkString("{", ",", "}")
|
||||
case (seq: Seq[_], ArrayType(typ, _)) =>
|
||||
seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
|
||||
|
|
|
@ -133,7 +133,7 @@ object ExtractPythonUDFs extends Rule[SparkPlan] with PredicateHelper {
|
|||
val validUdfs = udfs.filter { udf =>
|
||||
// Check to make sure that the UDF can be evaluated with only the input of this child.
|
||||
udf.references.subsetOf(child.outputSet)
|
||||
}.toArray // Turn it into an array since iterators cannot be serialized in Scala 2.10
|
||||
}.toArray
|
||||
if (validUdfs.nonEmpty) {
|
||||
val resultAttrs = udfs.zipWithIndex.map { case (u, i) =>
|
||||
AttributeReference(s"pythonUDF$i", u.dataType)()
|
||||
|
|
|
@ -359,7 +359,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
|
|||
|
||||
test("reduce") {
|
||||
val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
|
||||
assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == ("sum", 6))
|
||||
assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == (("sum", 6)))
|
||||
}
|
||||
|
||||
test("joinWith, flat schema") {
|
||||
|
@ -784,7 +784,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
|
|||
test("SPARK-14000: case class with tuple type field") {
|
||||
checkDataset(
|
||||
Seq(TupleClass((1, "a"))).toDS(),
|
||||
TupleClass(1, "a")
|
||||
TupleClass((1, "a"))
|
||||
)
|
||||
}
|
||||
|
||||
|
|
|
@ -176,16 +176,16 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared
|
|||
|
||||
test("number format in statistics") {
|
||||
val numbers = Seq(
|
||||
BigInt(0) -> ("0.0 B", "0"),
|
||||
BigInt(100) -> ("100.0 B", "100"),
|
||||
BigInt(2047) -> ("2047.0 B", "2.05E+3"),
|
||||
BigInt(2048) -> ("2.0 KB", "2.05E+3"),
|
||||
BigInt(3333333) -> ("3.2 MB", "3.33E+6"),
|
||||
BigInt(4444444444L) -> ("4.1 GB", "4.44E+9"),
|
||||
BigInt(5555555555555L) -> ("5.1 TB", "5.56E+12"),
|
||||
BigInt(6666666666666666L) -> ("5.9 PB", "6.67E+15"),
|
||||
BigInt(1L << 10 ) * (1L << 60) -> ("1024.0 EB", "1.18E+21"),
|
||||
BigInt(1L << 11) * (1L << 60) -> ("2.36E+21 B", "2.36E+21")
|
||||
BigInt(0) -> (("0.0 B", "0")),
|
||||
BigInt(100) -> (("100.0 B", "100")),
|
||||
BigInt(2047) -> (("2047.0 B", "2.05E+3")),
|
||||
BigInt(2048) -> (("2.0 KB", "2.05E+3")),
|
||||
BigInt(3333333) -> (("3.2 MB", "3.33E+6")),
|
||||
BigInt(4444444444L) -> (("4.1 GB", "4.44E+9")),
|
||||
BigInt(5555555555555L) -> (("5.1 TB", "5.56E+12")),
|
||||
BigInt(6666666666666666L) -> (("5.9 PB", "6.67E+15")),
|
||||
BigInt(1L << 10 ) * (1L << 60) -> (("1024.0 EB", "1.18E+21")),
|
||||
BigInt(1L << 11) * (1L << 60) -> (("2.36E+21 B", "2.36E+21"))
|
||||
)
|
||||
numbers.foreach { case (input, (expectedSize, expectedRows)) =>
|
||||
val stats = Statistics(sizeInBytes = input, rowCount = Some(input))
|
||||
|
|
|
@ -27,23 +27,23 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
|
|||
val row = identity[(java.lang.Integer, java.lang.Double)](_)
|
||||
|
||||
lazy val l = Seq(
|
||||
row(1, 2.0),
|
||||
row(1, 2.0),
|
||||
row(2, 1.0),
|
||||
row(2, 1.0),
|
||||
row(3, 3.0),
|
||||
row(null, null),
|
||||
row(null, 5.0),
|
||||
row(6, null)).toDF("a", "b")
|
||||
row((1, 2.0)),
|
||||
row((1, 2.0)),
|
||||
row((2, 1.0)),
|
||||
row((2, 1.0)),
|
||||
row((3, 3.0)),
|
||||
row((null, null)),
|
||||
row((null, 5.0)),
|
||||
row((6, null))).toDF("a", "b")
|
||||
|
||||
lazy val r = Seq(
|
||||
row(2, 3.0),
|
||||
row(2, 3.0),
|
||||
row(3, 2.0),
|
||||
row(4, 1.0),
|
||||
row(null, null),
|
||||
row(null, 5.0),
|
||||
row(6, null)).toDF("c", "d")
|
||||
row((2, 3.0)),
|
||||
row((2, 3.0)),
|
||||
row((3, 2.0)),
|
||||
row((4, 1.0)),
|
||||
row((null, null)),
|
||||
row((null, 5.0)),
|
||||
row((6, null))).toDF("c", "d")
|
||||
|
||||
lazy val t = r.filter($"c".isNotNull && $"d".isNotNull)
|
||||
|
||||
|
|
|
@ -211,7 +211,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
|
|||
}
|
||||
|
||||
testStandardAndLegacyModes("nested map with struct as value type") {
|
||||
val data = (1 to 4).map(i => Tuple1(Map(i -> (i, s"val_$i"))))
|
||||
val data = (1 to 4).map(i => Tuple1(Map(i -> ((i, s"val_$i")))))
|
||||
withParquetDataFrame(data) { df =>
|
||||
checkAnswer(df, data.map { case Tuple1(m) =>
|
||||
Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*)))
|
||||
|
|
|
@ -146,8 +146,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
// PhysicalRDD(nodeId = 1) -> Filter(nodeId = 0)
|
||||
val df = person.filter('age < 25)
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
0L -> ("Filter", Map(
|
||||
"number of output rows" -> 1L)))
|
||||
0L -> (("Filter", Map(
|
||||
"number of output rows" -> 1L))))
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -170,8 +170,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
Map("number of output rows" -> 1L,
|
||||
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
2L -> ("HashAggregate", expected1(0)),
|
||||
0L -> ("HashAggregate", expected1(1)))
|
||||
2L -> (("HashAggregate", expected1(0))),
|
||||
0L -> (("HashAggregate", expected1(1))))
|
||||
)
|
||||
|
||||
// 2 partitions and each partition contains 2 keys
|
||||
|
@ -182,8 +182,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
Map("number of output rows" -> 3L,
|
||||
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))
|
||||
testSparkPlanMetrics(df2, 1, Map(
|
||||
2L -> ("HashAggregate", expected2(0)),
|
||||
0L -> ("HashAggregate", expected2(1)))
|
||||
2L -> (("HashAggregate", expected2(0))),
|
||||
0L -> (("HashAggregate", expected2(1))))
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -234,15 +234,15 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
// -> ObjectHashAggregate(nodeId = 0)
|
||||
val df = testData2.groupBy().agg(collect_set('a)) // 2 partitions
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
2L -> ("ObjectHashAggregate", Map("number of output rows" -> 2L)),
|
||||
0L -> ("ObjectHashAggregate", Map("number of output rows" -> 1L)))
|
||||
2L -> (("ObjectHashAggregate", Map("number of output rows" -> 2L))),
|
||||
0L -> (("ObjectHashAggregate", Map("number of output rows" -> 1L))))
|
||||
)
|
||||
|
||||
// 2 partitions and each partition contains 2 keys
|
||||
val df2 = testData2.groupBy('a).agg(collect_set('a))
|
||||
testSparkPlanMetrics(df2, 1, Map(
|
||||
2L -> ("ObjectHashAggregate", Map("number of output rows" -> 4L)),
|
||||
0L -> ("ObjectHashAggregate", Map("number of output rows" -> 3L)))
|
||||
2L -> (("ObjectHashAggregate", Map("number of output rows" -> 4L))),
|
||||
0L -> (("ObjectHashAggregate", Map("number of output rows" -> 3L))))
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -264,9 +264,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
val df = spark.sql(
|
||||
"SELECT * FROM testData2 JOIN testDataForJoin ON testData2.a = testDataForJoin.a")
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
0L -> ("SortMergeJoin", Map(
|
||||
0L -> (("SortMergeJoin", Map(
|
||||
// It's 4 because we only read 3 rows in the first partition and 1 row in the second one
|
||||
"number of output rows" -> 4L)))
|
||||
"number of output rows" -> 4L))))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
@ -282,17 +282,17 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
val df = spark.sql(
|
||||
"SELECT * FROM testData2 left JOIN testDataForJoin ON testData2.a = testDataForJoin.a")
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
0L -> ("SortMergeJoin", Map(
|
||||
0L -> (("SortMergeJoin", Map(
|
||||
// It's 4 because we only read 3 rows in the first partition and 1 row in the second one
|
||||
"number of output rows" -> 8L)))
|
||||
"number of output rows" -> 8L))))
|
||||
)
|
||||
|
||||
val df2 = spark.sql(
|
||||
"SELECT * FROM testDataForJoin right JOIN testData2 ON testData2.a = testDataForJoin.a")
|
||||
testSparkPlanMetrics(df2, 1, Map(
|
||||
0L -> ("SortMergeJoin", Map(
|
||||
0L -> (("SortMergeJoin", Map(
|
||||
// It's 4 because we only read 3 rows in the first partition and 1 row in the second one
|
||||
"number of output rows" -> 8L)))
|
||||
"number of output rows" -> 8L))))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
@ -304,9 +304,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
// ... -> BroadcastHashJoin(nodeId = 1) -> TungstenProject(nodeId = 0)
|
||||
val df = df1.join(broadcast(df2), "key")
|
||||
testSparkPlanMetrics(df, 2, Map(
|
||||
1L -> ("BroadcastHashJoin", Map(
|
||||
1L -> (("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 2L,
|
||||
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))
|
||||
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))))
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -365,9 +365,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
val df = df1.join(df2, "key")
|
||||
val metrics = getSparkPlanMetrics(df, 1, Set(1L))
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
1L -> ("ShuffledHashJoin", Map(
|
||||
1L -> (("ShuffledHashJoin", Map(
|
||||
"number of output rows" -> 2L,
|
||||
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))
|
||||
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
@ -426,14 +426,14 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
// ... -> BroadcastHashJoin(nodeId = 0)
|
||||
val df = df1.join(broadcast(df2), $"key" === $"key2", "left_outer")
|
||||
testSparkPlanMetrics(df, 2, Map(
|
||||
0L -> ("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 5L)))
|
||||
0L -> (("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 5L))))
|
||||
)
|
||||
|
||||
val df3 = df1.join(broadcast(df2), $"key" === $"key2", "right_outer")
|
||||
testSparkPlanMetrics(df3, 2, Map(
|
||||
0L -> ("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 6L)))
|
||||
0L -> (("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 6L))))
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -448,8 +448,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
"SELECT * FROM testData2 left JOIN testDataForJoin ON " +
|
||||
"testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a")
|
||||
testSparkPlanMetrics(df, 3, Map(
|
||||
1L -> ("BroadcastNestedLoopJoin", Map(
|
||||
"number of output rows" -> 12L)))
|
||||
1L -> (("BroadcastNestedLoopJoin", Map(
|
||||
"number of output rows" -> 12L))))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
@ -462,8 +462,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
// ... -> BroadcastHashJoin(nodeId = 0)
|
||||
val df = df1.join(broadcast(df2), $"key" === $"key2", "leftsemi")
|
||||
testSparkPlanMetrics(df, 2, Map(
|
||||
0L -> ("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 2L)))
|
||||
0L -> (("BroadcastHashJoin", Map(
|
||||
"number of output rows" -> 2L))))
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -477,7 +477,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
val df = spark.sql(
|
||||
"SELECT * FROM testData2 JOIN testDataForJoin")
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
0L -> ("CartesianProduct", Map("number of output rows" -> 12L)))
|
||||
0L -> (("CartesianProduct", Map("number of output rows" -> 12L))))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
@ -490,7 +490,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext {
|
|||
val df = spark.sql(
|
||||
"SELECT * FROM testData2 ANTI JOIN antiData ON testData2.a = antiData.a")
|
||||
testSparkPlanMetrics(df, 1, Map(
|
||||
0L -> ("SortMergeJoin", Map("number of output rows" -> 4L)))
|
||||
0L -> (("SortMergeJoin", Map("number of output rows" -> 4L))))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -395,7 +395,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest
|
|||
}
|
||||
// Listener tracks only SQL metrics, not other accumulators
|
||||
assert(trackedAccums.size === 1)
|
||||
assert(trackedAccums.head === (sqlMetricInfo.id, sqlMetricInfo.update.get))
|
||||
assert(trackedAccums.head === ((sqlMetricInfo.id, sqlMetricInfo.update.get)))
|
||||
}
|
||||
|
||||
test("driver side SQL metrics") {
|
||||
|
|
|
@ -27,7 +27,7 @@ class ReduceAggregatorSuite extends SparkFunSuite {
|
|||
val encoder: ExpressionEncoder[Int] = ExpressionEncoder()
|
||||
val func = (v1: Int, v2: Int) => v1 + v2
|
||||
val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt)
|
||||
assert(aggregator.zero == (false, null))
|
||||
assert(aggregator.zero == (false, null).asInstanceOf[(Boolean, Int)])
|
||||
}
|
||||
|
||||
test("reduce, merge and finish") {
|
||||
|
@ -36,22 +36,22 @@ class ReduceAggregatorSuite extends SparkFunSuite {
|
|||
val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt)
|
||||
|
||||
val firstReduce = aggregator.reduce(aggregator.zero, 1)
|
||||
assert(firstReduce == (true, 1))
|
||||
assert(firstReduce == ((true, 1)))
|
||||
|
||||
val secondReduce = aggregator.reduce(firstReduce, 2)
|
||||
assert(secondReduce == (true, 3))
|
||||
assert(secondReduce == ((true, 3)))
|
||||
|
||||
val thirdReduce = aggregator.reduce(secondReduce, 3)
|
||||
assert(thirdReduce == (true, 6))
|
||||
assert(thirdReduce == ((true, 6)))
|
||||
|
||||
val mergeWithZero1 = aggregator.merge(aggregator.zero, firstReduce)
|
||||
assert(mergeWithZero1 == (true, 1))
|
||||
assert(mergeWithZero1 == ((true, 1)))
|
||||
|
||||
val mergeWithZero2 = aggregator.merge(secondReduce, aggregator.zero)
|
||||
assert(mergeWithZero2 == (true, 3))
|
||||
assert(mergeWithZero2 == ((true, 3)))
|
||||
|
||||
val mergeTwoReduced = aggregator.merge(firstReduce, secondReduce)
|
||||
assert(mergeTwoReduced == (true, 4))
|
||||
assert(mergeTwoReduced == ((true, 4)))
|
||||
|
||||
assert(aggregator.finish(firstReduce)== 1)
|
||||
assert(aggregator.finish(secondReduce) == 3)
|
||||
|
|
|
@ -20,12 +20,10 @@ package org.apache.spark.sql.streaming
|
|||
import java.util.UUID
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.language.reflectiveCalls
|
||||
|
||||
import org.scalactic.TolerantNumerics
|
||||
import org.scalatest.concurrent.AsyncAssertions.Waiter
|
||||
import org.scalatest.concurrent.Eventually._
|
||||
import org.scalatest.concurrent.PatienceConfiguration.Timeout
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import org.scalatest.PrivateMethodTester._
|
||||
|
|
|
@ -414,7 +414,7 @@ private[spark] object HiveUtils extends Logging {
|
|||
protected[sql] def toHiveString(a: (Any, DataType)): String = a match {
|
||||
case (struct: Row, StructType(fields)) =>
|
||||
struct.toSeq.zip(fields).map {
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}"""
|
||||
}.mkString("{", ",", "}")
|
||||
case (seq: Seq[_], ArrayType(typ, _)) =>
|
||||
seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
|
||||
|
@ -437,7 +437,7 @@ private[spark] object HiveUtils extends Logging {
|
|||
protected def toHiveStructString(a: (Any, DataType)): String = a match {
|
||||
case (struct: Row, StructType(fields)) =>
|
||||
struct.toSeq.zip(fields).map {
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
|
||||
case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}"""
|
||||
}.mkString("{", ",", "}")
|
||||
case (seq: Seq[_], ArrayType(typ, _)) =>
|
||||
seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
|
||||
|
|
|
@ -80,17 +80,17 @@ class DataSourceWithHiveMetastoreCatalogSuite
|
|||
).coalesce(1)
|
||||
|
||||
Seq(
|
||||
"parquet" -> (
|
||||
"parquet" -> ((
|
||||
"org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
|
||||
"org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
|
||||
"org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"
|
||||
),
|
||||
)),
|
||||
|
||||
"orc" -> (
|
||||
"orc" -> ((
|
||||
"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
|
||||
"org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
|
||||
"org.apache.hadoop.hive.ql.io.orc.OrcSerde"
|
||||
)
|
||||
))
|
||||
).foreach { case (provider, (inputFormat, outputFormat, serde)) =>
|
||||
test(s"Persist non-partitioned $provider relation into metastore as managed table") {
|
||||
withTable("t") {
|
||||
|
|
|
@ -151,7 +151,7 @@ class HiveSparkSubmitSuite
|
|||
// the HiveContext code mistakenly overrides the class loader that contains user classes.
|
||||
// For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala.
|
||||
val version = Properties.versionNumberString match {
|
||||
case v if v.startsWith("2.10") || v.startsWith("2.11") => v.substring(0, 4)
|
||||
case v if v.startsWith("2.12") || v.startsWith("2.11") => v.substring(0, 4)
|
||||
case x => throw new Exception(s"Unsupported Scala Version: $x")
|
||||
}
|
||||
val jarDir = getTestResourcePath("regression-test-SPARK-8489")
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.spark.sql.hive.execution
|
|||
import java.io.File
|
||||
import java.net.URI
|
||||
|
||||
import scala.language.existentials
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.scalatest.BeforeAndAfterEach
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue