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 16c6fdbe5274d..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" =>
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/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/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/pom.xml b/pom.xml
index 542efbaf06eb0..55d82ad298def 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 8b9d96e0b880e..244baf7639da9 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..18aaae93b05f2 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
@@ -251,6 +251,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 732b6855587dd..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
@@ -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 a52bfa59a1365..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)
}
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/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 19d4f34e565e9..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
@@ -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),
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..ddc435cf1a2e6 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.
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..0f7ae7a1c7de8 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
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/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)