diff --git a/bin/utils.sh b/bin/utils.sh
index 22ea2b9a6d586..2241200082018 100755
--- a/bin/utils.sh
+++ b/bin/utils.sh
@@ -26,14 +26,14 @@ function gatherSparkSubmitOpts() {
exit 1
fi
- # NOTE: If you add or remove spark-sumbmit options,
+ # NOTE: If you add or remove spark-submit options,
# modify NOT ONLY this script but also SparkSubmitArgument.scala
SUBMISSION_OPTS=()
APPLICATION_OPTS=()
while (($#)); do
case "$1" in
- --master | --deploy-mode | --class | --name | --jars | --py-files | --files | \
- --conf | --properties-file | --driver-memory | --driver-java-options | \
+ --master | --deploy-mode | --class | --name | --jars | --packages | --py-files | --files | \
+ --conf | --repositories | --properties-file | --driver-memory | --driver-java-options | \
--driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \
--total-executor-cores | --executor-cores | --queue | --num-executors | --archives)
if [[ $# -lt 2 ]]; then
diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd
index 1082a952dac99..567b8733f7f77 100644
--- a/bin/windows-utils.cmd
+++ b/bin/windows-utils.cmd
@@ -32,7 +32,7 @@ SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--p
SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<--driver-java-options\>"
SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>"
SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>"
-SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\>"
+SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\> \<--packages\> \<--repositories\>"
echo %1 | findstr %opts% >nul
if %ERRORLEVEL% equ 0 (
diff --git a/core/pom.xml b/core/pom.xml
index 2c115683fce66..2dc5f747f2b71 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -241,6 +241,17 @@
derby
test
+
+ org.apache.ivy
+ ivy
+ ${ivy.version}
+
+
+ oro
+
+ oro
+ ${oro.version}
+
org.tachyonproject
tachyon-client
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 6a16a31654630..7f5aef1c75df2 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -657,6 +657,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
*
* Load data from a flat binary file, assuming the length of each record is constant.
*
+ * '''Note:''' We ensure that the byte array for each record in the resulting RDD
+ * has the provided record length.
+ *
* @param path Directory to the input data files
* @param recordLength The length at which to split the records
* @return An RDD of data with values, represented as byte arrays
@@ -671,7 +674,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
classOf[LongWritable],
classOf[BytesWritable],
conf=conf)
- val data = br.map{ case (k, v) => v.getBytes}
+ val data = br.map { case (k, v) =>
+ val bytes = v.getBytes
+ assert(bytes.length == recordLength, "Byte array does not have correct length")
+ bytes
+ }
data
}
@@ -1224,7 +1231,19 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
null
}
} else {
- env.httpFileServer.addJar(new File(uri.getPath))
+ try {
+ env.httpFileServer.addJar(new File(uri.getPath))
+ } catch {
+ case exc: FileNotFoundException =>
+ logError(s"Jar not found at $path")
+ null
+ case e: Exception =>
+ // For now just log an error but allow to go through so spark examples work.
+ // The spark examples don't really need the jar distributed since its also
+ // the app jar.
+ logError("Error adding jar (" + e + "), was the --addJars option used?")
+ null
+ }
}
// A JAR file which exists locally on every worker node
case "local" =>
@@ -1749,8 +1768,14 @@ object SparkContext extends Logging {
@deprecated("Replaced by implicit functions in the RDD companion object. This is " +
"kept here only for backward compatibility.", "1.3.0")
def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
- rdd: RDD[(K, V)]) =
+ rdd: RDD[(K, V)]) = {
+ val kf = implicitly[K => Writable]
+ val vf = implicitly[V => Writable]
+ // Set the Writable class to null and `SequenceFileRDDFunctions` will use Reflection to get it
+ implicit val keyWritableFactory = new WritableFactory[K](_ => null, kf)
+ implicit val valueWritableFactory = new WritableFactory[V](_ => null, vf)
RDD.rddToSequenceFileRDDFunctions(rdd)
+ }
@deprecated("Replaced by implicit functions in the RDD companion object. This is " +
"kept here only for backward compatibility.", "1.3.0")
@@ -1767,20 +1792,35 @@ object SparkContext extends Logging {
def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) =
RDD.numericRDDToDoubleRDDFunctions(rdd)
- // Implicit conversions to common Writable types, for saveAsSequenceFile
+ // The following deprecated functions have already been moved to `object WritableFactory` to
+ // make the compiler find them automatically. They are still kept here for backward compatibility.
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def intToIntWritable(i: Int): IntWritable = new IntWritable(i)
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def longToLongWritable(l: Long): LongWritable = new LongWritable(l)
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def floatToFloatWritable(f: Float): FloatWritable = new FloatWritable(f)
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def doubleToDoubleWritable(d: Double): DoubleWritable = new DoubleWritable(d)
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def boolToBoolWritable (b: Boolean): BooleanWritable = new BooleanWritable(b)
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def bytesToBytesWritable (aob: Array[Byte]): BytesWritable = new BytesWritable(aob)
+ @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " +
+ "kept here only for backward compatibility.", "1.3.0")
implicit def stringToText(s: String): Text = new Text(s)
private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T])
@@ -2070,7 +2110,7 @@ object WritableConverter {
new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W]))
}
- // The following implicit functions were in SparkContext before 1.2 and users had to
+ // The following implicit functions were in SparkContext before 1.3 and users had to
// `import SparkContext._` to enable them. Now we move them here to make the compiler find
// them automatically. However, we still keep the old functions in SparkContext for backward
// compatibility and forward to the following functions directly.
@@ -2103,3 +2143,46 @@ object WritableConverter {
implicit def writableWritableConverter[T <: Writable](): WritableConverter[T] =
new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T])
}
+
+/**
+ * A class encapsulating how to convert some type T to Writable. It stores both the Writable class
+ * corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion.
+ * The Writable class will be used in `SequenceFileRDDFunctions`.
+ */
+private[spark] class WritableFactory[T](
+ val writableClass: ClassTag[T] => Class[_ <: Writable],
+ val convert: T => Writable) extends Serializable
+
+object WritableFactory {
+
+ private[spark] def simpleWritableFactory[T: ClassTag, W <: Writable : ClassTag](convert: T => W)
+ : WritableFactory[T] = {
+ val writableClass = implicitly[ClassTag[W]].runtimeClass.asInstanceOf[Class[W]]
+ new WritableFactory[T](_ => writableClass, convert)
+ }
+
+ implicit def intWritableFactory: WritableFactory[Int] =
+ simpleWritableFactory(new IntWritable(_))
+
+ implicit def longWritableFactory: WritableFactory[Long] =
+ simpleWritableFactory(new LongWritable(_))
+
+ implicit def floatWritableFactory: WritableFactory[Float] =
+ simpleWritableFactory(new FloatWritable(_))
+
+ implicit def doubleWritableFactory: WritableFactory[Double] =
+ simpleWritableFactory(new DoubleWritable(_))
+
+ implicit def booleanWritableFactory: WritableFactory[Boolean] =
+ simpleWritableFactory(new BooleanWritable(_))
+
+ implicit def bytesWritableFactory: WritableFactory[Array[Byte]] =
+ simpleWritableFactory(new BytesWritable(_))
+
+ implicit def stringWritableFactory: WritableFactory[String] =
+ simpleWritableFactory(new Text(_))
+
+ implicit def writableWritableFactory[T <: Writable: ClassTag]: WritableFactory[T] =
+ simpleWritableFactory(w => w)
+
+}
diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
index c0cbd28a845be..cf289fb3ae39f 100644
--- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
@@ -107,7 +107,6 @@ private[python] class WritableToDoubleArrayConverter extends Converter[Any, Arra
* given directory (probably a temp directory)
*/
object WriteInputFormatTestDataGenerator {
- import SparkContext._
def main(args: Array[String]) {
val path = args(0)
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 02021be9f93d4..8bbfcd2997dc6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -25,6 +25,17 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, Map}
import org.apache.hadoop.fs.Path
+import org.apache.ivy.Ivy
+import org.apache.ivy.core.LogOptions
+import org.apache.ivy.core.module.descriptor.{DefaultExcludeRule, DefaultDependencyDescriptor, DefaultModuleDescriptor}
+import org.apache.ivy.core.module.id.{ModuleId, ArtifactId, ModuleRevisionId}
+import org.apache.ivy.core.report.ResolveReport
+import org.apache.ivy.core.resolve.{IvyNode, ResolveOptions}
+import org.apache.ivy.core.retrieve.RetrieveOptions
+import org.apache.ivy.core.settings.IvySettings
+import org.apache.ivy.plugins.matcher.GlobPatternMatcher
+import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver}
+
import org.apache.spark.executor.ExecutorURLClassLoader
import org.apache.spark.util.Utils
@@ -194,6 +205,18 @@ object SparkSubmit {
// Special flag to avoid deprecation warnings at the client
sysProps("SPARK_SUBMIT") = "true"
+ // Resolve maven dependencies if there are any and add classpath to jars
+ val resolvedMavenCoordinates =
+ SparkSubmitUtils.resolveMavenCoordinates(
+ args.packages, Option(args.repositories), Option(args.ivyRepoPath))
+ if (!resolvedMavenCoordinates.trim.isEmpty) {
+ if (args.jars == null || args.jars.trim.isEmpty) {
+ args.jars = resolvedMavenCoordinates
+ } else {
+ args.jars += s",$resolvedMavenCoordinates"
+ }
+ }
+
// A list of rules to map each argument to system properties or command-line options in
// each deploy mode; we iterate through these below
val options = List[OptionAssigner](
@@ -202,6 +225,7 @@ object SparkSubmit {
OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"),
OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"),
OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"),
+ OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars.ivy"),
OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT,
sysProp = "spark.driver.memory"),
OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
@@ -213,6 +237,7 @@ object SparkSubmit {
// Standalone cluster only
OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"),
+ OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"),
OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"),
OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"),
@@ -384,8 +409,8 @@ object SparkSubmit {
case e: ClassNotFoundException =>
e.printStackTrace(printStream)
if (childMainClass.contains("thriftserver")) {
- println(s"Failed to load main class $childMainClass.")
- println("You need to build Spark with -Phive and -Phive-thriftserver.")
+ printStream.println(s"Failed to load main class $childMainClass.")
+ printStream.println("You need to build Spark with -Phive and -Phive-thriftserver.")
}
System.exit(CLASS_NOT_FOUND_EXIT_STATUS)
}
@@ -475,6 +500,194 @@ object SparkSubmit {
}
}
+/** Provides utility functions to be used inside SparkSubmit. */
+private[spark] object SparkSubmitUtils {
+
+ // Exposed for testing
+ private[spark] var printStream = SparkSubmit.printStream
+
+ /**
+ * Represents a Maven Coordinate
+ * @param groupId the groupId of the coordinate
+ * @param artifactId the artifactId of the coordinate
+ * @param version the version of the coordinate
+ */
+ private[spark] case class MavenCoordinate(groupId: String, artifactId: String, version: String)
+
+/**
+ * Extracts maven coordinates from a comma-delimited string
+ * @param coordinates Comma-delimited string of maven coordinates
+ * @return Sequence of Maven coordinates
+ */
+ private[spark] def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = {
+ coordinates.split(",").map { p =>
+ val splits = p.split(":")
+ require(splits.length == 3, s"Provided Maven Coordinates must be in the form " +
+ s"'groupId:artifactId:version'. The coordinate provided is: $p")
+ require(splits(0) != null && splits(0).trim.nonEmpty, s"The groupId cannot be null or " +
+ s"be whitespace. The groupId provided is: ${splits(0)}")
+ require(splits(1) != null && splits(1).trim.nonEmpty, s"The artifactId cannot be null or " +
+ s"be whitespace. The artifactId provided is: ${splits(1)}")
+ require(splits(2) != null && splits(2).trim.nonEmpty, s"The version cannot be null or " +
+ s"be whitespace. The version provided is: ${splits(2)}")
+ new MavenCoordinate(splits(0), splits(1), splits(2))
+ }
+ }
+
+ /**
+ * Extracts maven coordinates from a comma-delimited string
+ * @param remoteRepos Comma-delimited string of remote repositories
+ * @return A ChainResolver used by Ivy to search for and resolve dependencies.
+ */
+ private[spark] def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = {
+ // We need a chain resolver if we want to check multiple repositories
+ val cr = new ChainResolver
+ cr.setName("list")
+
+ // the biblio resolver resolves POM declared dependencies
+ val br: IBiblioResolver = new IBiblioResolver
+ br.setM2compatible(true)
+ br.setUsepoms(true)
+ br.setName("central")
+ cr.add(br)
+
+ val repositoryList = remoteRepos.getOrElse("")
+ // add any other remote repositories other than maven central
+ if (repositoryList.trim.nonEmpty) {
+ repositoryList.split(",").zipWithIndex.foreach { case (repo, i) =>
+ val brr: IBiblioResolver = new IBiblioResolver
+ brr.setM2compatible(true)
+ brr.setUsepoms(true)
+ brr.setRoot(repo)
+ brr.setName(s"repo-${i + 1}")
+ cr.add(brr)
+ printStream.println(s"$repo added as a remote repository with the name: ${brr.getName}")
+ }
+ }
+ cr
+ }
+
+ /**
+ * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath
+ * (will append to jars in SparkSubmit). The name of the jar is given
+ * after a '!' by Ivy. It also sometimes contains '(bundle)' after '.jar'. Remove that as well.
+ * @param artifacts Sequence of dependencies that were resolved and retrieved
+ * @param cacheDirectory directory where jars are cached
+ * @return a comma-delimited list of paths for the dependencies
+ */
+ private[spark] def resolveDependencyPaths(
+ artifacts: Array[AnyRef],
+ cacheDirectory: File): String = {
+ artifacts.map { artifactInfo =>
+ val artifactString = artifactInfo.toString
+ val jarName = artifactString.drop(artifactString.lastIndexOf("!") + 1)
+ cacheDirectory.getAbsolutePath + File.separator +
+ jarName.substring(0, jarName.lastIndexOf(".jar") + 4)
+ }.mkString(",")
+ }
+
+ /** Adds the given maven coordinates to Ivy's module descriptor. */
+ private[spark] def addDependenciesToIvy(
+ md: DefaultModuleDescriptor,
+ artifacts: Seq[MavenCoordinate],
+ ivyConfName: String): Unit = {
+ artifacts.foreach { mvn =>
+ val ri = ModuleRevisionId.newInstance(mvn.groupId, mvn.artifactId, mvn.version)
+ val dd = new DefaultDependencyDescriptor(ri, false, false)
+ dd.addDependencyConfiguration(ivyConfName, ivyConfName)
+ printStream.println(s"${dd.getDependencyId} added as a dependency")
+ md.addDependency(dd)
+ }
+ }
+
+ /** A nice function to use in tests as well. Values are dummy strings. */
+ private[spark] def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance(
+ ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0"))
+
+ /**
+ * Resolves any dependencies that were supplied through maven coordinates
+ * @param coordinates Comma-delimited string of maven coordinates
+ * @param remoteRepos Comma-delimited string of remote repositories other than maven central
+ * @param ivyPath The path to the local ivy repository
+ * @return The comma-delimited path to the jars of the given maven artifacts including their
+ * transitive dependencies
+ */
+ private[spark] def resolveMavenCoordinates(
+ coordinates: String,
+ remoteRepos: Option[String],
+ ivyPath: Option[String],
+ isTest: Boolean = false): String = {
+ if (coordinates == null || coordinates.trim.isEmpty) {
+ ""
+ } else {
+ val artifacts = extractMavenCoordinates(coordinates)
+ // Default configuration name for ivy
+ val ivyConfName = "default"
+ // set ivy settings for location of cache
+ val ivySettings: IvySettings = new IvySettings
+ // Directories for caching downloads through ivy and storing the jars when maven coordinates
+ // are supplied to spark-submit
+ val alternateIvyCache = ivyPath.getOrElse("")
+ val packagesDirectory: File =
+ if (alternateIvyCache.trim.isEmpty) {
+ new File(ivySettings.getDefaultIvyUserDir, "jars")
+ } else {
+ ivySettings.setDefaultCache(new File(alternateIvyCache, "cache"))
+ new File(alternateIvyCache, "jars")
+ }
+ printStream.println(
+ s"Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}")
+ printStream.println(s"The jars for the packages stored in: $packagesDirectory")
+ // create a pattern matcher
+ ivySettings.addMatcher(new GlobPatternMatcher)
+ // create the dependency resolvers
+ val repoResolver = createRepoResolvers(remoteRepos)
+ ivySettings.addResolver(repoResolver)
+ ivySettings.setDefaultResolver(repoResolver.getName)
+
+ val ivy = Ivy.newInstance(ivySettings)
+ // Set resolve options to download transitive dependencies as well
+ val resolveOptions = new ResolveOptions
+ resolveOptions.setTransitive(true)
+ val retrieveOptions = new RetrieveOptions
+ // Turn downloading and logging off for testing
+ if (isTest) {
+ resolveOptions.setDownload(false)
+ resolveOptions.setLog(LogOptions.LOG_QUIET)
+ retrieveOptions.setLog(LogOptions.LOG_QUIET)
+ } else {
+ resolveOptions.setDownload(true)
+ }
+
+ // A Module descriptor must be specified. Entries are dummy strings
+ val md = getModuleDescriptor
+ md.setDefaultConf(ivyConfName)
+
+ // Add an exclusion rule for Spark
+ val sparkArtifacts = new ArtifactId(new ModuleId("org.apache.spark", "*"), "*", "*", "*")
+ val sparkDependencyExcludeRule =
+ new DefaultExcludeRule(sparkArtifacts, ivySettings.getMatcher("glob"), null)
+ sparkDependencyExcludeRule.addConfiguration(ivyConfName)
+
+ // Exclude any Spark dependencies, and add all supplied maven artifacts as dependencies
+ md.addExcludeRule(sparkDependencyExcludeRule)
+ addDependenciesToIvy(md, artifacts, ivyConfName)
+
+ // resolve dependencies
+ val rr: ResolveReport = ivy.resolve(md, resolveOptions)
+ if (rr.hasError) {
+ throw new RuntimeException(rr.getAllProblemMessages.toString)
+ }
+ // retrieve all resolved dependencies
+ ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId,
+ packagesDirectory.getAbsolutePath + File.separator + "[artifact](-[classifier]).[ext]",
+ retrieveOptions.setConfs(Array(ivyConfName)))
+
+ resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory)
+ }
+ }
+}
+
/**
* Provides an indirection layer for passing arguments as system properties or flags to
* the user's driver program or to downstream launcher tools.
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index 73e921fd83ef2..5cadc534f4baa 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -50,6 +50,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
var name: String = null
var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]()
var jars: String = null
+ var packages: String = null
+ var repositories: String = null
+ var ivyRepoPath: String = null
var verbose: Boolean = false
var isPython: Boolean = false
var pyFiles: String = null
@@ -123,6 +126,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
.orNull
name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull
jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull
+ ivyRepoPath = sparkProperties.get("spark.jars.ivy").orNull
deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull
numExecutors = Option(numExecutors)
.getOrElse(sparkProperties.get("spark.executor.instances").orNull)
@@ -212,6 +216,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
| name $name
| childArgs [${childArgs.mkString(" ")}]
| jars $jars
+ | packages $packages
+ | repositories $repositories
| verbose $verbose
|
|Spark properties used, including those specified through
@@ -318,6 +324,14 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
jars = Utils.resolveURIs(value)
parse(tail)
+ case ("--packages") :: value :: tail =>
+ packages = value
+ parse(tail)
+
+ case ("--repositories") :: value :: tail =>
+ repositories = value
+ parse(tail)
+
case ("--conf" | "-c") :: value :: tail =>
value.split("=", 2).toSeq match {
case Seq(k, v) => sparkProperties(k) = v
@@ -368,6 +382,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
| --name NAME A name of your application.
| --jars JARS Comma-separated list of local jars to include on the driver
| and executor classpaths.
+ | --packages Comma-separated list of maven coordinates of jars to include
+ | on the driver and executor classpaths. Will search the local
+ | maven repo, then maven central and any additional remote
+ | repositories given by --repositories. The format for the
+ | coordinates should be groupId:artifactId:version.
+ | --repositories Comma-separated list of additional remote repositories to
+ | search for the maven coordinates given with --packages.
| --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to place
| on the PYTHONPATH for Python apps.
| --files FILES Comma-separated list of files to be placed in the working
diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala
index 5ad73c3d27f47..b6249b492150a 100644
--- a/core/src/main/scala/org/apache/spark/package.scala
+++ b/core/src/main/scala/org/apache/spark/package.scala
@@ -27,8 +27,7 @@ package org.apache
* contains operations available only on RDDs of Doubles; and
* [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can
* be saved as SequenceFiles. These operations are automatically available on any RDD of the right
- * type (e.g. RDD[(Int, Int)] through implicit conversions except `saveAsSequenceFile`. You need to
- * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work.
+ * type (e.g. RDD[(Int, Int)] through implicit conversions.
*
* Java programmers should reference the [[org.apache.spark.api.java]] package
* for Spark programming APIs in Java.
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 97aee58bddbf1..fe55a5124f3b6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -25,11 +25,8 @@ import scala.language.implicitConversions
import scala.reflect.{classTag, ClassTag}
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus
-import org.apache.hadoop.io.BytesWritable
+import org.apache.hadoop.io.{Writable, BytesWritable, NullWritable, Text}
import org.apache.hadoop.io.compress.CompressionCodec
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.TextOutputFormat
import org.apache.spark._
@@ -57,8 +54,7 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli
* [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that
* can be saved as SequenceFiles.
* All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)]
- * through implicit conversions except `saveAsSequenceFile`. You need to
- * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work.
+ * through implicit.
*
* Internally, each RDD is characterized by five main properties:
*
@@ -1527,7 +1523,7 @@ abstract class RDD[T: ClassTag](
*/
object RDD {
- // The following implicit functions were in SparkContext before 1.2 and users had to
+ // The following implicit functions were in SparkContext before 1.3 and users had to
// `import SparkContext._` to enable them. Now we move them here to make the compiler find
// them automatically. However, we still keep the old functions in SparkContext for backward
// compatibility and forward to the following functions directly.
@@ -1541,9 +1537,15 @@ object RDD {
new AsyncRDDActions(rdd)
}
- implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
- rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = {
- new SequenceFileRDDFunctions(rdd)
+ implicit def rddToSequenceFileRDDFunctions[K, V](rdd: RDD[(K, V)])
+ (implicit kt: ClassTag[K], vt: ClassTag[V],
+ keyWritableFactory: WritableFactory[K],
+ valueWritableFactory: WritableFactory[V])
+ : SequenceFileRDDFunctions[K, V] = {
+ implicit val keyConverter = keyWritableFactory.convert
+ implicit val valueConverter = valueWritableFactory.convert
+ new SequenceFileRDDFunctions(rdd,
+ keyWritableFactory.writableClass(kt), valueWritableFactory.writableClass(vt))
}
implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](rdd: RDD[(K, V)])
diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
index 2b48916951430..059f8963691f0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
@@ -30,13 +30,35 @@ import org.apache.spark.Logging
* through an implicit conversion. Note that this can't be part of PairRDDFunctions because
* we need more implicit parameters to convert our keys and values to Writable.
*
- * Import `org.apache.spark.SparkContext._` at the top of their program to use these functions.
*/
class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag](
- self: RDD[(K, V)])
+ self: RDD[(K, V)],
+ _keyWritableClass: Class[_ <: Writable],
+ _valueWritableClass: Class[_ <: Writable])
extends Logging
with Serializable {
+ @deprecated("It's used to provide backward compatibility for pre 1.3.0.", "1.3.0")
+ def this(self: RDD[(K, V)]) {
+ this(self, null, null)
+ }
+
+ 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)) {
@@ -55,6 +77,7 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
c.asInstanceOf[Class[_ <: Writable]]
}
+
/**
* Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key
* and value types. If the key or value are Writable, then we use their classes directly;
@@ -65,26 +88,28 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) {
def anyToWritable[U <% Writable](u: U): Writable = u
- val keyClass = getWritableClass[K]
- val valueClass = getWritableClass[V]
- val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass)
- val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass)
+ // TODO We cannot force the return type of `anyToWritable` be same as keyWritableClass and
+ // 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
- logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," +
- valueClass.getSimpleName + ")" )
+ logInfo("Saving as sequence file of type (" + keyWritableClass.getSimpleName + "," +
+ valueWritableClass.getSimpleName + ")" )
val format = classOf[SequenceFileOutputFormat[Writable, Writable]]
val jobConf = new JobConf(self.context.hadoopConfiguration)
if (!convertKey && !convertValue) {
- self.saveAsHadoopFile(path, keyClass, valueClass, 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, keyClass, valueClass, format, jobConf, codec)
+ path, keyWritableClass, valueWritableClass, format, jobConf, codec)
} else if (convertKey && !convertValue) {
self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(
- path, keyClass, valueClass, format, jobConf, codec)
+ path, keyWritableClass, valueWritableClass, format, jobConf, codec)
} else if (convertKey && convertValue) {
self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(
- path, keyClass, valueClass, format, jobConf, codec)
+ path, keyWritableClass, valueWritableClass, format, jobConf, codec)
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
index 05b6fa54564b7..4676b828d3d89 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
@@ -19,6 +19,8 @@ package org.apache.spark.scheduler.local
import java.nio.ByteBuffer
+import scala.concurrent.duration._
+
import akka.actor.{Actor, ActorRef, Props}
import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState}
@@ -46,6 +48,8 @@ private[spark] class LocalActor(
private val totalCores: Int)
extends Actor with ActorLogReceive with Logging {
+ import context.dispatcher // to use Akka's scheduler.scheduleOnce()
+
private var freeCores = totalCores
private val localExecutorId = SparkContext.DRIVER_IDENTIFIER
@@ -74,11 +78,16 @@ private[spark] class LocalActor(
def reviveOffers() {
val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores))
- for (task <- scheduler.resourceOffers(offers).flatten) {
+ val tasks = scheduler.resourceOffers(offers).flatten
+ for (task <- tasks) {
freeCores -= scheduler.CPUS_PER_TASK
executor.launchTask(executorBackend, taskId = task.taskId, attemptNumber = task.attemptNumber,
task.name, task.serializedTask)
}
+ if (tasks.isEmpty && scheduler.activeTaskSets.nonEmpty) {
+ // Try to reviveOffer after 1 second, because scheduler may wait for locality timeout
+ context.system.scheduler.scheduleOnce(1000 millis, self, ReviveOffers)
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index e9f2aed9ffbea..3c8a0e40bf785 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -279,7 +279,7 @@ private[spark] object Utils extends Logging {
maxAttempts + " attempts!")
}
try {
- dir = new File(root, "spark-" + UUID.randomUUID.toString)
+ dir = new File(root, namePrefix + "-" + UUID.randomUUID.toString)
if (dir.exists() || !dir.mkdirs()) {
dir = null
} else {
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index 5e24196101fbc..7acd27c735727 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInp
import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat}
import org.scalatest.FunSuite
-import org.apache.spark.SparkContext._
import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD}
import org.apache.spark.util.Utils
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 82628ad3abd99..3f1355f82893e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -307,7 +307,21 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"--name", "testApp",
"--master", "local-cluster[2,1,512]",
"--jars", jarsString,
- unusedJar.toString)
+ unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB")
+ runSparkSubmit(args)
+ }
+
+ test("includes jars passed in through --packages") {
+ val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
+ val packagesString = "com.databricks:spark-csv_2.10:0.1,com.databricks:spark-avro_2.10:0.1"
+ val args = Seq(
+ "--class", JarCreationTest.getClass.getName.stripSuffix("$"),
+ "--name", "testApp",
+ "--master", "local-cluster[2,1,512]",
+ "--packages", packagesString,
+ "--conf", "spark.ui.enabled=false",
+ unusedJar.toString,
+ "com.databricks.spark.csv.DefaultSource", "com.databricks.spark.avro.DefaultSource")
runSparkSubmit(args)
}
@@ -467,8 +481,8 @@ object JarCreationTest extends Logging {
val result = sc.makeRDD(1 to 100, 10).mapPartitions { x =>
var exception: String = null
try {
- Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader)
- Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader)
+ Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
+ Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
} catch {
case t: Throwable =>
exception = t + "\n" + t.getStackTraceString
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
new file mode 100644
index 0000000000000..53665350013cd
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
@@ -0,0 +1,121 @@
+/*
+ * 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.deploy
+
+import java.io.{PrintStream, OutputStream, File}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+
+import org.apache.ivy.core.module.descriptor.MDArtifact
+import org.apache.ivy.plugins.resolver.IBiblioResolver
+
+class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll {
+
+ private val noOpOutputStream = new OutputStream {
+ def write(b: Int) = {}
+ }
+
+ /** Simple PrintStream that reads data into a buffer */
+ private class BufferPrintStream extends PrintStream(noOpOutputStream) {
+ var lineBuffer = ArrayBuffer[String]()
+ override def println(line: String) {
+ lineBuffer += line
+ }
+ }
+
+ override def beforeAll() {
+ super.beforeAll()
+ // We don't want to write logs during testing
+ SparkSubmitUtils.printStream = new BufferPrintStream
+ }
+
+ test("incorrect maven coordinate throws error") {
+ val coordinates = Seq("a:b: ", " :a:b", "a: :b", "a:b:", ":a:b", "a::b", "::", "a:b", "a")
+ for (coordinate <- coordinates) {
+ intercept[IllegalArgumentException] {
+ SparkSubmitUtils.extractMavenCoordinates(coordinate)
+ }
+ }
+ }
+
+ test("create repo resolvers") {
+ val resolver1 = SparkSubmitUtils.createRepoResolvers(None)
+ // should have central by default
+ assert(resolver1.getResolvers.size() === 1)
+ assert(resolver1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "central")
+
+ val repos = "a/1,b/2,c/3"
+ val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos))
+ assert(resolver2.getResolvers.size() === 4)
+ val expected = repos.split(",").map(r => s"$r/")
+ resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: IBiblioResolver, i) =>
+ if (i == 0) {
+ assert(resolver.getName === "central")
+ } else {
+ assert(resolver.getName === s"repo-$i")
+ assert(resolver.getRoot === expected(i - 1))
+ }
+ }
+ }
+
+ 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")
+
+ SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default")
+ assert(md.getDependencies.length === 2)
+ }
+
+ test("ivy path works correctly") {
+ val ivyPath = "dummy/ivy"
+ val md = SparkSubmitUtils.getModuleDescriptor
+ val artifacts = for (i <- 0 until 3) yield new MDArtifact(md, s"jar-$i", "jar", "jar")
+ var jPaths = SparkSubmitUtils.resolveDependencyPaths(artifacts.toArray, new File(ivyPath))
+ for (i <- 0 until 3) {
+ val index = jPaths.indexOf(ivyPath)
+ assert(index >= 0)
+ jPaths = jPaths.substring(index + ivyPath.length)
+ }
+ // end to end
+ val jarPath = SparkSubmitUtils.resolveMavenCoordinates(
+ "com.databricks:spark-csv_2.10:0.1", None, Option(ivyPath), true)
+ assert(jarPath.indexOf(ivyPath) >= 0, "should use non-default ivy path")
+ }
+
+ test("search for artifact at other repositories") {
+ val path = SparkSubmitUtils.resolveMavenCoordinates("com.agimatec:agimatec-validation:0.9.3",
+ Option("https://oss.sonatype.org/content/repositories/agimatec/"), None, true)
+ assert(path.indexOf("agimatec-validation") >= 0, "should find package. If it doesn't, check" +
+ "if package still exists. If it has been removed, replace the example in this test.")
+ }
+
+ test("dependency not found throws RuntimeException") {
+ intercept[RuntimeException] {
+ SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, true)
+ }
+ }
+
+ test("neglects Spark and Spark's dependencies") {
+ val path = SparkSubmitUtils.resolveMavenCoordinates(
+ "org.apache.spark:spark-core_2.10:1.2.0", None, None, true)
+ assert(path === "", "should return empty path")
+ }
+}
diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala
index 4918e2d92beb4..daa795a043495 100644
--- a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala
+++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala
@@ -44,13 +44,21 @@ class ImplicitSuite {
}
def testRddToSequenceFileRDDFunctions(): Unit = {
- // TODO eliminating `import intToIntWritable` needs refactoring SequenceFileRDDFunctions.
- // That will be a breaking change.
- import org.apache.spark.SparkContext.intToIntWritable
val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD
rdd.saveAsSequenceFile("/a/test/path")
}
+ def testRddToSequenceFileRDDFunctionsWithWritable(): Unit = {
+ val rdd: org.apache.spark.rdd.RDD[(org.apache.hadoop.io.IntWritable, org.apache.hadoop.io.Text)]
+ = mockRDD
+ rdd.saveAsSequenceFile("/a/test/path")
+ }
+
+ def testRddToSequenceFileRDDFunctionsWithBytesArray(): Unit = {
+ val rdd: org.apache.spark.rdd.RDD[(Int, Array[Byte])] = mockRDD
+ rdd.saveAsSequenceFile("/a/test/path")
+ }
+
def testRddToOrderedRDDFunctions(): Unit = {
val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD
rdd.sortByKey()
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
index 265827b3341c2..f2c38e79c452c 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
@@ -40,7 +40,7 @@ class ShortestPathsSuite extends FunSuite with LocalSparkContext {
val graph = Graph.fromEdgeTuples(edges, 1)
val landmarks = Seq(1, 4).map(_.toLong)
val results = ShortestPaths.run(graph, landmarks).vertices.collect.map {
- case (v, spMap) => (v, spMap.mapValues(_.get))
+ case (v, spMap) => (v, spMap.mapValues(i => i))
}
assert(results.toSet === shortestPaths)
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
index 18be35ad59452..df90078de148f 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
@@ -132,14 +132,14 @@ class LogisticRegressionModel private[ml] (
override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
transformSchema(dataset.schema, paramMap, logging = true)
val map = this.paramMap ++ paramMap
- val scoreFunction: Vector => Double = (v) => {
+ val scoreFunction = udf((v: Vector) => {
val margin = BLAS.dot(v, weights)
1.0 / (1.0 + math.exp(-margin))
- }
+ } : Double)
val t = map(threshold)
- val predictFunction: Double => Double = (score) => { if (score > t) 1.0 else 0.0 }
+ val predictFunction = udf((score: Double) => { if (score > t) 1.0 else 0.0 } : Double)
dataset
- .select($"*", callUDF(scoreFunction, col(map(featuresCol))).as(map(scoreCol)))
- .select($"*", callUDF(predictFunction, col(map(scoreCol))).as(map(predictionCol)))
+ .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol)))
+ .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol)))
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
index 01a4f5eb205e5..4745a7ae95679 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
@@ -81,10 +81,8 @@ class StandardScalerModel private[ml] (
override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
transformSchema(dataset.schema, paramMap, logging = true)
val map = this.paramMap ++ paramMap
- val scale: (Vector) => Vector = (v) => {
- scaler.transform(v)
- }
- dataset.select($"*", callUDF(scale, col(map(inputCol))).as(map(outputCol)))
+ val scale = udf((v: Vector) => { scaler.transform(v) } : Vector)
+ dataset.select($"*", scale(col(map(inputCol))).as(map(outputCol)))
}
private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = {
diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
index 511cb2fe4005e..c7bec7a845a11 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
@@ -126,22 +126,20 @@ class ALSModel private[ml] (
val map = this.paramMap ++ paramMap
val users = userFactors.toDataFrame("id", "features")
val items = itemFactors.toDataFrame("id", "features")
- val predict: (Seq[Float], Seq[Float]) => Float = (userFeatures, itemFeatures) => {
+
+ // Register a UDF for DataFrame, and then
+ // create a new column named map(predictionCol) by running the predict UDF.
+ val predict = udf((userFeatures: Seq[Float], itemFeatures: Seq[Float]) => {
if (userFeatures != null && itemFeatures != null) {
blas.sdot(k, userFeatures.toArray, 1, itemFeatures.toArray, 1)
} else {
Float.NaN
}
- }
- val inputColumns = dataset.schema.fieldNames
- val prediction = callUDF(predict, users("features"), items("features")).as(map(predictionCol))
- val outputColumns = inputColumns.map(f => dataset(f)) :+ prediction
+ } : Float)
dataset
.join(users, dataset(map(userCol)) === users("id"), "left")
.join(items, dataset(map(itemCol)) === items("id"), "left")
- .select(outputColumns: _*)
- // TODO: Just use a dataset("*")
- // .select(dataset("*"), prediction)
+ .select(dataset("*"), predict(users("features"), items("features")).as(map(predictionCol)))
}
override private[ml] def transformSchema(schema: StructType, paramMap: ParamMap): StructType = {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index 980980593d194..3f29b82ce85ac 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -295,7 +295,7 @@ class PythonMLLibAPI extends Serializable {
k: Int,
convergenceTol: Double,
maxIterations: Int,
- seed: Long): JList[Object] = {
+ seed: java.lang.Long): JList[Object] = {
val gmmAlg = new GaussianMixture()
.setK(k)
.setConvergenceTol(convergenceTol)
diff --git a/pom.xml b/pom.xml
index 75d217a8c0349..021c11d422284 100644
--- a/pom.xml
+++ b/pom.xml
@@ -136,6 +136,8 @@
1.2.3
8.1.14.v20131031
0.5.0
+ 2.4.0
+ 2.0.8
3.1.0
1.7.6
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index 268c7ef97cffc..74305dea749c8 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -2126,10 +2126,9 @@ def sort(self, *cols):
"""
if not cols:
raise ValueError("should sort by at least one column")
- jcols = ListConverter().convert([_to_java_column(c) for c in cols[1:]],
+ jcols = ListConverter().convert([_to_java_column(c) for c in cols],
self._sc._gateway._gateway_client)
- jdf = self._jdf.sort(_to_java_column(cols[0]),
- self._sc._jvm.Dsl.toColumns(jcols))
+ jdf = self._jdf.sort(self._sc._jvm.Dsl.toColumns(jcols))
return DataFrame(jdf, self.sql_ctx)
sortBy = sort
diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py
index d48f3598e33b2..b06ab650370bd 100644
--- a/python/pyspark/streaming/context.py
+++ b/python/pyspark/streaming/context.py
@@ -21,7 +21,7 @@
from py4j.java_gateway import java_import, JavaObject
from pyspark import RDD, SparkConf
-from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
+from pyspark.serializers import NoOpSerializer, UTF8Deserializer, CloudPickleSerializer
from pyspark.context import SparkContext
from pyspark.storagelevel import StorageLevel
from pyspark.streaming.dstream import DStream
@@ -191,6 +191,15 @@ def awaitTermination(self, timeout=None):
else:
self._jssc.awaitTermination(int(timeout * 1000))
+ def awaitTerminationOrTimeout(self, timeout):
+ """
+ Wait for the execution to stop. Return `true` if it's stopped; or
+ throw the reported error during the execution; or `false` if the
+ waiting time elapsed before returning from the method.
+ @param timeout: time to wait in seconds
+ """
+ self._jssc.awaitTerminationOrTimeout(int(timeout * 1000))
+
def stop(self, stopSparkContext=True, stopGraceFully=False):
"""
Stop the execution of the streams, with option of ensuring all
@@ -251,6 +260,20 @@ def textFileStream(self, directory):
"""
return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer())
+ def binaryRecordsStream(self, directory, recordLength):
+ """
+ Create an input stream that monitors a Hadoop-compatible file system
+ for new files and reads them as flat binary files with records of
+ fixed length. Files must be written to the monitored directory by "moving"
+ them from another location within the same file system.
+ File names starting with . are ignored.
+
+ @param directory: Directory to load data from
+ @param recordLength: Length of each record in bytes
+ """
+ return DStream(self._jssc.binaryRecordsStream(directory, recordLength), self,
+ NoOpSerializer())
+
def _check_serializers(self, rdds):
# make sure they have same serializer
if len(set(rdd._jrdd_deserializer for rdd in rdds)) > 1:
diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py
index a8d876d0fa3b3..608f8e26473a6 100644
--- a/python/pyspark/streaming/tests.py
+++ b/python/pyspark/streaming/tests.py
@@ -21,6 +21,7 @@
import operator
import unittest
import tempfile
+import struct
from pyspark.context import SparkConf, SparkContext, RDD
from pyspark.streaming.context import StreamingContext
@@ -455,6 +456,20 @@ def test_text_file_stream(self):
self.wait_for(result, 2)
self.assertEqual([range(10), range(10)], result)
+ def test_binary_records_stream(self):
+ d = tempfile.mkdtemp()
+ self.ssc = StreamingContext(self.sc, self.duration)
+ dstream = self.ssc.binaryRecordsStream(d, 10).map(
+ lambda v: struct.unpack("10b", str(v)))
+ result = self._collect(dstream, 2, block=False)
+ self.ssc.start()
+ for name in ('a', 'b'):
+ time.sleep(1)
+ with open(os.path.join(d, name), "wb") as f:
+ f.write(bytearray(range(10)))
+ self.wait_for(result, 2)
+ self.assertEqual([range(10), range(10)], map(lambda v: list(v[0]), result))
+
def test_union(self):
input = [range(i + 1) for i in range(3)]
dstream = self.ssc.queueStream(input)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 5c006e9d4c6f5..a9bd079c7049d 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -36,6 +36,16 @@ import org.apache.spark.sql.types._
* for a SQL like language should checkout the HiveQL support in the sql/hive sub-project.
*/
class SqlParser extends AbstractSparkSQLParser {
+
+ def parseExpression(input: String): Expression = {
+ // Initialize the Keywords.
+ lexical.initialize(reservedWords)
+ phrase(expression)(new lexical.Scanner(input)) match {
+ case Success(plan, _) => plan
+ case failureOrError => sys.error(failureOrError.toString)
+ }
+ }
+
// Keyword is a convention with AbstractSparkSQLParser, which will scan all of the `Keyword`
// properties via reflection the class in runtime for constructing the SqlLexical object
protected val ABS = Keyword("ABS")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index f3bc07ae5238c..a4997fb293781 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -54,10 +54,10 @@ private[sql] object DataFrame {
* }}}
*
* Note that the [[Column]] type can also be manipulated through its various functions.
- * {{
+ * {{{
* // The following creates a new column that increases everybody's age by 10.
* people("age") + 10 // in Scala
- * }}
+ * }}}
*
* A more concrete example:
* {{{
@@ -173,7 +173,7 @@ trait DataFrame extends RDDApi[Row] {
* }}}
*/
@scala.annotation.varargs
- def sort(sortExpr: Column, sortExprs: Column*): DataFrame
+ def sort(sortExprs: Column*): DataFrame
/**
* Returns a new [[DataFrame]] sorted by the given expressions.
@@ -187,7 +187,7 @@ trait DataFrame extends RDDApi[Row] {
* This is an alias of the `sort` function.
*/
@scala.annotation.varargs
- def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame
+ def orderBy(sortExprs: Column*): DataFrame
/**
* Selects column based on the column name and return it as a [[Column]].
@@ -236,6 +236,17 @@ trait DataFrame extends RDDApi[Row] {
@scala.annotation.varargs
def select(col: String, cols: String*): DataFrame
+ /**
+ * Selects a set of SQL expressions. This is a variant of `select` that accepts
+ * SQL expressions.
+ *
+ * {{{
+ * df.selectExpr("colA", "colB as newName", "abs(colC)")
+ * }}}
+ */
+ @scala.annotation.varargs
+ def selectExpr(exprs: String*): DataFrame
+
/**
* Filters rows using the given condition.
* {{{
@@ -247,6 +258,14 @@ trait DataFrame extends RDDApi[Row] {
*/
def filter(condition: Column): DataFrame
+ /**
+ * Filters rows using the given SQL expression.
+ * {{{
+ * peopleDf.filter("age > 15")
+ * }}}
+ */
+ def filter(conditionExpr: String): DataFrame
+
/**
* Filters rows using the given condition. This is an alias for `filter`.
* {{{
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
index 0b0623dc1fe75..c702adcb65122 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
@@ -27,7 +27,7 @@ import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.{SqlParser, ScalaReflection}
import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.{JoinType, Inner}
@@ -124,11 +124,11 @@ private[sql] class DataFrameImpl protected[sql](
}
override def sort(sortCol: String, sortCols: String*): DataFrame = {
- orderBy(apply(sortCol), sortCols.map(apply) :_*)
+ sort((sortCol +: sortCols).map(apply) :_*)
}
- override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = {
- val sortOrder: Seq[SortOrder] = (sortExpr +: sortExprs).map { col =>
+ override def sort(sortExprs: Column*): DataFrame = {
+ val sortOrder: Seq[SortOrder] = sortExprs.map { col =>
col.expr match {
case expr: SortOrder =>
expr
@@ -143,8 +143,8 @@ private[sql] class DataFrameImpl protected[sql](
sort(sortCol, sortCols :_*)
}
- override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = {
- sort(sortExpr, sortExprs :_*)
+ override def orderBy(sortExprs: Column*): DataFrame = {
+ sort(sortExprs :_*)
}
override def col(colName: String): Column = colName match {
@@ -179,10 +179,20 @@ private[sql] class DataFrameImpl protected[sql](
select((col +: cols).map(Column(_)) :_*)
}
+ override def selectExpr(exprs: String*): DataFrame = {
+ select(exprs.map { expr =>
+ Column(new SqlParser().parseExpression(expr))
+ } :_*)
+ }
+
override def filter(condition: Column): DataFrame = {
Filter(condition.expr, logicalPlan)
}
+ override def filter(conditionExpr: String): DataFrame = {
+ filter(Column(new SqlParser().parseExpression(conditionExpr)))
+ }
+
override def where(condition: Column): DataFrame = {
filter(condition)
}
@@ -329,7 +339,7 @@ private[sql] class DataFrameImpl protected[sql](
override def save(path: String): Unit = {
val dataSourceName = sqlContext.conf.defaultDataSourceName
- save(dataSourceName, ("path" -> path))
+ save(dataSourceName, "path" -> path)
}
override def save(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
index 71365c776d559..8cf59f0a1f099 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
@@ -186,15 +186,13 @@ object Dsl {
(0 to 22).map { x =>
val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"})
val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _)
- val args = (1 to x).map(i => s"arg$i: Column").mkString(", ")
- val argsInUdf = (1 to x).map(i => s"arg$i.expr").mkString(", ")
println(s"""
/**
- * Call a Scala function of ${x} arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of ${x} arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[$typeTags](f: Function$x[$types]${if (args.length > 0) ", " + args else ""}): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq($argsInUdf))
+ def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}""")
}
@@ -214,187 +212,187 @@ object Dsl {
}
*/
/**
- * Call a Scala function of 0 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 0 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag](f: Function0[RT]): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq())
+ def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 1 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 1 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT], arg1: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr))
+ def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 2 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 2 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT], arg1: Column, arg2: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 3 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 3 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT], arg1: Column, arg2: Column, arg3: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 4 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 4 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 5 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 5 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 6 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 6 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 7 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 7 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 8 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 8 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 9 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 9 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 10 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 10 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 11 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 11 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](f: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](f: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 12 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 12 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](f: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](f: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 13 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 13 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](f: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](f: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 14 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 14 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](f: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](f: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 15 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 15 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](f: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](f: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 16 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 16 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](f: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](f: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 17 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 17 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](f: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](f: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 18 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 18 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](f: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](f: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 19 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 19 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](f: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](f: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 20 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 20 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](f: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](f: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 21 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 21 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](f: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](f: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
/**
- * Call a Scala function of 22 arguments as user-defined function (UDF), and automatically
- * infer the data types based on the function's signature.
+ * Defines a user-defined function of 22 arguments as user-defined function (UDF).
+ * The data types are automatically inferred based on the function's signature.
*/
- def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](f: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column, arg22: Column): Column = {
- ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr, arg22.expr))
+ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](f: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = {
+ UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType)
}
//////////////////////////////////////////////////////////////////////////////////////////////////
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala
index ba5c7355b4b70..6b032d3d699a9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala
@@ -66,11 +66,11 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten
override def sort(sortCol: String, sortCols: String*): DataFrame = err()
- override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = err()
+ override def sort(sortExprs: Column*): DataFrame = err()
override def orderBy(sortCol: String, sortCols: String*): DataFrame = err()
- override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = err()
+ override def orderBy(sortExprs: Column*): DataFrame = err()
override def col(colName: String): Column = err()
@@ -80,8 +80,12 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten
override def select(col: String, cols: String*): DataFrame = err()
+ override def selectExpr(exprs: String*): DataFrame = err()
+
override def filter(condition: Column): DataFrame = err()
+ override def filter(conditionExpr: String): DataFrame = err()
+
override def where(condition: Column): DataFrame = err()
override def apply(condition: Column): DataFrame = err()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
similarity index 56%
rename from sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
index 1beb19437a8da..d8b0a3b26dbab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
@@ -78,20 +78,21 @@ class UDFRegistration(sqlContext: SQLContext) extends Logging {
// scalastyle:off
- /* registerFunction 0-22 were generated by this script
+ /* register 0-22 were generated by this script
(0 to 22).map { x =>
val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"})
val typeTags = (1 to x).map(i => s"A${i}: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _)
- val argDocs = (1 to x).map(i => s" * @tparam A$i type of the UDF argument at position $i.").foldLeft("")(_ + "\n" + _)
println(s"""
/**
* Register a Scala closure of ${x} arguments as user-defined function (UDF).
- * @tparam RT return type of UDF.$argDocs
+ * @tparam RT return type of UDF.
*/
- def register[$typeTags](name: String, func: Function$x[$types]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}""")
}
@@ -116,462 +117,258 @@ class UDFRegistration(sqlContext: SQLContext) extends Logging {
* Register a Scala closure of 0 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
*/
- def register[RT: TypeTag](name: String, func: Function0[RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 1 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
*/
- def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 2 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
*/
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 3 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
*/
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 4 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
*/
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 5 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 6 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 7 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 8 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 9 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 10 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 11 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 12 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 13 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 14 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 15 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 16 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 17 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- * @tparam A17 type of the UDF argument at position 17.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 18 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- * @tparam A17 type of the UDF argument at position 17.
- * @tparam A18 type of the UDF argument at position 18.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 19 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- * @tparam A17 type of the UDF argument at position 17.
- * @tparam A18 type of the UDF argument at position 18.
- * @tparam A19 type of the UDF argument at position 19.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 20 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- * @tparam A17 type of the UDF argument at position 17.
- * @tparam A18 type of the UDF argument at position 18.
- * @tparam A19 type of the UDF argument at position 19.
- * @tparam A20 type of the UDF argument at position 20.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 21 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- * @tparam A17 type of the UDF argument at position 17.
- * @tparam A18 type of the UDF argument at position 18.
- * @tparam A19 type of the UDF argument at position 19.
- * @tparam A20 type of the UDF argument at position 20.
- * @tparam A21 type of the UDF argument at position 21.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
/**
* Register a Scala closure of 22 arguments as user-defined function (UDF).
* @tparam RT return type of UDF.
- * @tparam A1 type of the UDF argument at position 1.
- * @tparam A2 type of the UDF argument at position 2.
- * @tparam A3 type of the UDF argument at position 3.
- * @tparam A4 type of the UDF argument at position 4.
- * @tparam A5 type of the UDF argument at position 5.
- * @tparam A6 type of the UDF argument at position 6.
- * @tparam A7 type of the UDF argument at position 7.
- * @tparam A8 type of the UDF argument at position 8.
- * @tparam A9 type of the UDF argument at position 9.
- * @tparam A10 type of the UDF argument at position 10.
- * @tparam A11 type of the UDF argument at position 11.
- * @tparam A12 type of the UDF argument at position 12.
- * @tparam A13 type of the UDF argument at position 13.
- * @tparam A14 type of the UDF argument at position 14.
- * @tparam A15 type of the UDF argument at position 15.
- * @tparam A16 type of the UDF argument at position 16.
- * @tparam A17 type of the UDF argument at position 17.
- * @tparam A18 type of the UDF argument at position 18.
- * @tparam A19 type of the UDF argument at position 19.
- * @tparam A20 type of the UDF argument at position 20.
- * @tparam A21 type of the UDF argument at position 21.
- * @tparam A22 type of the UDF argument at position 22.
- */
- def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): Unit = {
- def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e)
+ */
+ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = {
+ val dataType = ScalaReflection.schemaFor[RT].dataType
+ def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e)
functionRegistry.registerFunction(name, builder)
+ UserDefinedFunction(func, dataType)
}
+ //////////////////////////////////////////////////////////////////////////////////////////////
+ //////////////////////////////////////////////////////////////////////////////////////////////
+
/**
* Register a user-defined function with 1 arguments.
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala
new file mode 100644
index 0000000000000..8d7c2a1b8339e
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala
@@ -0,0 +1,39 @@
+/*
+* 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
+
+import org.apache.spark.sql.catalyst.expressions.ScalaUdf
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A user-defined function. To create one, use the `udf` functions in [[Dsl]].
+ * As an example:
+ * {{{
+ * // Defined a UDF that returns true or false based on some numeric score.
+ * val predict = udf((score: Double) => if (score > 0.5) true else false)
+ *
+ * // Projects a column that adds a prediction column based on the score column.
+ * df.select( predict(df("score")) )
+ * }}}
+ */
+case class UserDefinedFunction(f: AnyRef, dataType: DataType) {
+
+ def apply(exprs: Column*): Column = {
+ Column(ScalaUdf(f, dataType, exprs.map(_.expr)))
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index f6b65a81ce05e..e588555ad01d4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.Dsl._
import org.apache.spark.sql.types._
/* Implicits */
-import org.apache.spark.sql.test.TestSQLContext._
+import org.apache.spark.sql.test.TestSQLContext.{createDataFrame, logicalPlanToSparkQuery}
import scala.language.postfixOps
@@ -47,6 +47,18 @@ class DataFrameSuite extends QueryTest {
testData.collect().toSeq)
}
+ test("selectExpr") {
+ checkAnswer(
+ testData.selectExpr("abs(key)", "value"),
+ testData.collect().map(row => Row(math.abs(row.getInt(0)), row.getString(1))).toSeq)
+ }
+
+ test("filterExpr") {
+ checkAnswer(
+ testData.filter("key > 90"),
+ testData.collect().filter(_.getInt(0) > 90).toSeq)
+ }
+
test("repartition") {
checkAnswer(
testData.select('key).repartition(10).select('key),
@@ -280,11 +292,11 @@ class DataFrameSuite extends QueryTest {
}
test("udf") {
- val foo = (a: Int, b: String) => a.toString + b
+ val foo = udf((a: Int, b: String) => a.toString + b)
checkAnswer(
// SELECT *, foo(key, value) FROM testData
- testData.select($"*", callUDF(foo, 'key, 'value)).limit(3),
+ testData.select($"*", foo('key, 'value)).limit(3),
Row(1, "1", "11") :: Row(2, "2", "22") :: Row(3, "3", "33") :: Nil
)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 0501b47f080d7..8f3d4265a286d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql
import java.util.TimeZone
+import org.apache.spark.sql.test.TestSQLContext
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.sql.Dsl._
@@ -26,9 +27,8 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types._
-/* Implicits */
import org.apache.spark.sql.TestData._
-import org.apache.spark.sql.test.TestSQLContext._
+import org.apache.spark.sql.test.TestSQLContext.{udf => _, _}
class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
@@ -794,7 +794,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
}
test("SPARK-3371 Renaming a function expression with group by gives error") {
- udf.register("len", (s: String) => s.length)
+ TestSQLContext.udf.register("len", (s: String) => s.length)
checkAnswer(
sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"),
Row(1))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index 0696a2335e63f..117a51173481d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -21,7 +21,8 @@ import scala.beans.{BeanInfo, BeanProperty}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Dsl._
-import org.apache.spark.sql.test.TestSQLContext._
+import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.sql.test.TestSQLContext.{udf => _, _}
import org.apache.spark.sql.types._
@@ -83,7 +84,7 @@ class UserDefinedTypeSuite extends QueryTest {
}
test("UDTs and UDFs") {
- udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector])
+ TestSQLContext.udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector])
pointsRDD.registerTempTable("points")
checkAnswer(
sql("SELECT testType(features) from points"),
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 8ef0787137845..ba3f23434f24c 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -27,10 +27,12 @@ import scala.reflect.ClassTag
import akka.actor.{Props, SupervisorStrategy}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.{LongWritable, Text}
+import org.apache.hadoop.io.{BytesWritable, LongWritable, Text}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
import org.apache.spark._
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.input.FixedLengthBinaryInputFormat
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream._
@@ -359,6 +361,30 @@ class StreamingContext private[streaming] (
new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
}
+ /**
+ * Create a input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them using the given key-value types and input format.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system. File names starting with . are ignored.
+ * @param directory HDFS directory to monitor for new file
+ * @param filter Function to filter paths to process
+ * @param newFilesOnly Should process only new files and ignore existing files in the directory
+ * @param conf Hadoop configuration
+ * @tparam K Key type for reading HDFS file
+ * @tparam V Value type for reading HDFS file
+ * @tparam F Input format for reading HDFS file
+ */
+ def fileStream[
+ K: ClassTag,
+ V: ClassTag,
+ F <: NewInputFormat[K, V]: ClassTag
+ ] (directory: String,
+ filter: Path => Boolean,
+ newFilesOnly: Boolean,
+ conf: Configuration): InputDStream[(K, V)] = {
+ new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly, Option(conf))
+ }
+
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them as text files (using key as LongWritable, value
@@ -371,6 +397,37 @@ class StreamingContext private[streaming] (
fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
}
+ /**
+ * :: Experimental ::
+ *
+ * Create an input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them as flat binary files, assuming a fixed length per record,
+ * generating one byte array per record. Files must be written to the monitored directory
+ * by "moving" them from another location within the same file system. File names
+ * starting with . are ignored.
+ *
+ * '''Note:''' We ensure that the byte array for each record in the
+ * resulting RDDs of the DStream has the provided record length.
+ *
+ * @param directory HDFS directory to monitor for new file
+ * @param recordLength length of each record in bytes
+ */
+ @Experimental
+ def binaryRecordsStream(
+ directory: String,
+ recordLength: Int): DStream[Array[Byte]] = {
+ val conf = sc_.hadoopConfiguration
+ conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength)
+ val br = fileStream[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](
+ directory, FileInputDStream.defaultFilter : Path => Boolean, newFilesOnly=true, conf)
+ val data = br.map { case (k, v) =>
+ val bytes = v.getBytes
+ assert(bytes.length == recordLength, "Byte array does not have correct length")
+ bytes
+ }
+ data
+ }
+
/**
* Create an input stream from a queue of RDDs. In each batch,
* it will process either one or all of the RDDs returned by the queue.
@@ -469,10 +526,23 @@ class StreamingContext private[streaming] (
* will be thrown in this thread.
* @param timeout time to wait in milliseconds
*/
+ @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0")
def awaitTermination(timeout: Long) {
waiter.waitForStopOrError(timeout)
}
+ /**
+ * Wait for the execution to stop. Any exceptions that occurs during the execution
+ * will be thrown in this thread.
+ *
+ * @param timeout time to wait in milliseconds
+ * @return `true` if it's stopped; or throw the reported error during the execution; or `false`
+ * if the waiting time elapsed before returning from the method.
+ */
+ def awaitTerminationOrTimeout(timeout: Long): Boolean = {
+ waiter.waitForStopOrError(timeout)
+ }
+
/**
* Stop the execution of the streams immediately (does not wait for all received data
* to be processed).
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 9a2254bcdc1f7..e3db01c1e12c6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.annotation.Experimental
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
import org.apache.spark.rdd.RDD
@@ -177,7 +178,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
/**
* Create an input stream from network source hostname:port. Data is received using
- * a TCP socket and the receive bytes it interepreted as object using the given
+ * a TCP socket and the receive bytes it interpreted as object using the given
* converter.
* @param hostname Hostname to connect to for receiving data
* @param port Port to connect to for receiving data
@@ -209,6 +210,24 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
ssc.textFileStream(directory)
}
+ /**
+ * :: Experimental ::
+ *
+ * Create an input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them as flat binary files with fixed record lengths,
+ * yielding byte arrays
+ *
+ * '''Note:''' We ensure that the byte array for each record in the
+ * resulting RDDs of the DStream has the provided record length.
+ *
+ * @param directory HDFS directory to monitor for new files
+ * @param recordLength The length at which to split the records
+ */
+ @Experimental
+ def binaryRecordsStream(directory: String, recordLength: Int): JavaDStream[Array[Byte]] = {
+ ssc.binaryRecordsStream(directory, recordLength)
+ }
+
/**
* Create an input stream from network source hostname:port, where data is received
* as serialized blocks (serialized using the Spark's serializer) that can be directly
@@ -298,6 +317,37 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
ssc.fileStream[K, V, F](directory, fn, newFilesOnly)
}
+ /**
+ * Create an input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them using the given key-value types and input format.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system. File names starting with . are ignored.
+ * @param directory HDFS directory to monitor for new file
+ * @param kClass class of key for reading HDFS file
+ * @param vClass class of value for reading HDFS file
+ * @param fClass class of input format for reading HDFS file
+ * @param filter Function to filter paths to process
+ * @param newFilesOnly Should process only new files and ignore existing files in the directory
+ * @param conf Hadoop configuration
+ * @tparam K Key type for reading HDFS file
+ * @tparam V Value type for reading HDFS file
+ * @tparam F Input format for reading HDFS file
+ */
+ def fileStream[K, V, F <: NewInputFormat[K, V]](
+ directory: String,
+ kClass: Class[K],
+ vClass: Class[V],
+ fClass: Class[F],
+ filter: JFunction[Path, JBoolean],
+ newFilesOnly: Boolean,
+ conf: Configuration): JavaPairInputDStream[K, V] = {
+ implicit val cmk: ClassTag[K] = ClassTag(kClass)
+ implicit val cmv: ClassTag[V] = ClassTag(vClass)
+ implicit val cmf: ClassTag[F] = ClassTag(fClass)
+ def fn = (x: Path) => filter.call(x).booleanValue()
+ ssc.fileStream[K, V, F](directory, fn, newFilesOnly, conf)
+ }
+
/**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
@@ -547,10 +597,23 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
* will be thrown in this thread.
* @param timeout time to wait in milliseconds
*/
+ @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0")
def awaitTermination(timeout: Long): Unit = {
ssc.awaitTermination(timeout)
}
+ /**
+ * Wait for the execution to stop. Any exceptions that occurs during the execution
+ * will be thrown in this thread.
+ *
+ * @param timeout time to wait in milliseconds
+ * @return `true` if it's stopped; or throw the reported error during the execution; or `false`
+ * if the waiting time elapsed before returning from the method.
+ */
+ def awaitTerminationOrTimeout(timeout: Long): Boolean = {
+ ssc.awaitTerminationOrTimeout(timeout)
+ }
+
/**
* Stop the execution of the streams. Will stop the associated JavaSparkContext as well.
*/
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index e7c5639a63499..6379b88527ec8 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap
import scala.collection.mutable
import scala.reflect.ClassTag
+import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
@@ -68,11 +69,13 @@ import org.apache.spark.util.{TimeStampedHashMap, Utils}
* processing semantics are undefined.
*/
private[streaming]
-class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag](
+class FileInputDStream[K, V, F <: NewInputFormat[K,V]](
@transient ssc_ : StreamingContext,
directory: String,
filter: Path => Boolean = FileInputDStream.defaultFilter,
- newFilesOnly: Boolean = true)
+ newFilesOnly: Boolean = true,
+ conf: Option[Configuration] = None)
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F])
extends InputDStream[(K, V)](ssc_) {
// This is a def so that it works during checkpoint recovery:
@@ -237,7 +240,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
/** Generate one RDD from an array of files */
private def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
val fileRDDs = files.map(file =>{
- val rdd = context.sparkContext.newAPIHadoopFile[K, V, F](file)
+ val rdd = conf match {
+ case Some(config) => context.sparkContext.newAPIHadoopFile(
+ file,
+ fm.runtimeClass.asInstanceOf[Class[F]],
+ km.runtimeClass.asInstanceOf[Class[K]],
+ vm.runtimeClass.asInstanceOf[Class[V]],
+ config)
+ case None => context.sparkContext.newAPIHadoopFile[K, V, F](file)
+ }
if (rdd.partitions.size == 0) {
logError("File " + file + " has no data in it. Spark Streaming can only ingest " +
"files that have been \"moved\" to the directory assigned to the file stream. " +
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
index 00456ab2a0c92..b36aeb341d25e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
@@ -254,7 +254,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
// Check if all the receivers have been deregistered or not
if (!receiverInfo.isEmpty) {
- logWarning("All of the receivers have not deregistered, " + receiverInfo)
+ logWarning("Not all of the receivers have deregistered, " + receiverInfo)
} else {
logInfo("All of the receivers have deregistered successfully")
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index bddf51e130422..01084a457db4f 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -95,6 +95,57 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
}
}
+ test("binary records stream") {
+ val testDir: File = null
+ try {
+ val batchDuration = Seconds(2)
+ val testDir = Utils.createTempDir()
+ // Create a file that exists before the StreamingContext is created:
+ val existingFile = new File(testDir, "0")
+ Files.write("0\n", existingFile, Charset.forName("UTF-8"))
+ assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000)
+
+ // Set up the streaming context and input streams
+ withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc =>
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ // This `setTime` call ensures that the clock is past the creation time of `existingFile`
+ clock.setTime(existingFile.lastModified + batchDuration.milliseconds)
+ val batchCounter = new BatchCounter(ssc)
+ val fileStream = ssc.binaryRecordsStream(testDir.toString, 1)
+ val outputBuffer = new ArrayBuffer[Seq[Array[Byte]]]
+ with SynchronizedBuffer[Seq[Array[Byte]]]
+ val outputStream = new TestOutputStream(fileStream, outputBuffer)
+ outputStream.register()
+ ssc.start()
+
+ // Advance the clock so that the files are created after StreamingContext starts, but
+ // not enough to trigger a batch
+ clock.addToTime(batchDuration.milliseconds / 2)
+
+ val input = Seq(1, 2, 3, 4, 5)
+ input.foreach { i =>
+ Thread.sleep(batchDuration.milliseconds)
+ val file = new File(testDir, i.toString)
+ Files.write(Array[Byte](i.toByte), file)
+ assert(file.setLastModified(clock.currentTime()))
+ assert(file.lastModified === clock.currentTime)
+ logInfo("Created file " + file)
+ // Advance the clock after creating the file to avoid a race when
+ // setting its modification time
+ clock.addToTime(batchDuration.milliseconds)
+ eventually(eventuallyTimeout) {
+ assert(batchCounter.getNumCompletedBatches === i)
+ }
+ }
+
+ val expectedOutput = input.map(i => i.toByte)
+ val obtainedOutput = outputBuffer.flatten.toList.map(i => i(0).toByte)
+ assert(obtainedOutput === expectedOutput)
+ }
+ } finally {
+ if (testDir != null) Utils.deleteRecursively(testDir)
+ }
+ }
test("file input stream - newFilesOnly = true") {
testFileStream(newFilesOnly = true)
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
index 0b5af25e0f7cc..2aa5e0876b6e0 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
@@ -304,6 +304,30 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
assert(exception.getMessage.contains("transform"), "Expected exception not thrown")
}
+ test("awaitTerminationOrTimeout") {
+ ssc = new StreamingContext(master, appName, batchDuration)
+ val inputStream = addInputStream(ssc)
+ inputStream.map(x => x).register()
+
+ ssc.start()
+
+ // test whether awaitTerminationOrTimeout() return false after give amount of time
+ failAfter(1000 millis) {
+ assert(ssc.awaitTerminationOrTimeout(500) === false)
+ }
+
+ // test whether awaitTerminationOrTimeout() return true if context is stopped
+ failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown
+ new Thread() {
+ override def run() {
+ Thread.sleep(500)
+ ssc.stop()
+ }
+ }.start()
+ assert(ssc.awaitTerminationOrTimeout(10000) === true)
+ }
+ }
+
test("DStream and generated RDD creation sites") {
testPackage.test()
}