From 6dc5f5849c0e0378abc6648c919412827d831641 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Apr 2014 01:03:33 -0700 Subject: [PATCH 01/12] [SPARK-1396] Properly cleanup DAGScheduler on job cancellation. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, when jobs were cancelled, not all of the state in the DAGScheduler was cleaned up, leading to a slow memory leak in the DAGScheduler. As we expose easier ways to cancel jobs, it's more important to fix these issues. This commit also fixes a second and less serious problem, which is that previously, when a stage failed, not all of the appropriate stages were cancelled. See the "failure of stage used by two jobs" test for an example of this. This just meant that extra work was done, and is not a correctness problem. This commit adds 3 tests. “run shuffle with map stage failure” is a new test to more thoroughly test this functionality, and passes on both the old and new versions of the code. “trivial job cancellation” fails on the old code because all state wasn’t cleaned up correctly when jobs were cancelled (we didn’t remove the job from resultStageToJob). “failure of stage used by two jobs” fails on the old code because taskScheduler.cancelTasks wasn’t called for one of the stages (see test comments). This should be checked in before #246, which makes it easier to cancel stages / jobs. Author: Kay Ousterhout Closes #305 from kayousterhout/incremental_abort_fix and squashes the following commits: f33d844 [Kay Ousterhout] Mark review comments 9217080 [Kay Ousterhout] Properly cleanup DAGScheduler on job cancellation. --- .../apache/spark/scheduler/DAGScheduler.scala | 44 +++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 92 ++++++++++++++++++- 2 files changed, 115 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6368665f249ee..c96d7435a7ed4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -982,15 +982,7 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach(taskScheduler.cancelTasks) - val error = new SparkException("Job %d cancelled".format(jobId)) - val job = jobIdToActiveJob(jobId) - job.listener.jobFailed(error) - jobIdToStageIds -= jobId - activeJobs -= job - jobIdToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled") } } @@ -1007,19 +999,39 @@ class DAGScheduler( stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - val error = new SparkException("Job aborted: " + reason) - job.listener.jobFailed(error) - jobIdToStageIdsRemove(job.jobId) - jobIdToActiveJob -= resultStage.jobId - activeJobs -= job - resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } + /** + * Fails a job and all stages that are only used by that job, and cleans up relevant state. + */ + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { + val error = new SparkException(failureReason) + job.listener.jobFailed(error) + + // Cancel all tasks in independent stages. + val independentStages = removeJobAndIndependentStages(job.jobId) + independentStages.foreach(taskScheduler.cancelTasks) + + // Clean up remaining state we store for the job. + jobIdToActiveJob -= job.jobId + activeJobs -= job + jobIdToStageIds -= job.jobId + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob + + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + } + /** * Return true if one of stage's ancestors is target. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ce567b0cde85d..2e3026bffba2f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import scala.Tuple2 -import scala.collection.mutable.{HashMap, Map} +import scala.collection.mutable.{HashSet, HashMap, Map} import org.scalatest.{BeforeAndAfter, FunSuite} @@ -43,6 +43,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() + + /** Stages for which the DAGScheduler has called TaskScheduler.cancelTasks(). */ + val cancelledStages = new HashSet[Int]() + val taskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE @@ -53,7 +57,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) {} + override def cancelTasks(stageId: Int) { + cancelledStages += stageId + } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 } @@ -91,6 +97,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont before { sc = new SparkContext("local", "DAGSchedulerSuite") taskSets.clear() + cancelledStages.clear() cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -174,15 +181,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } } - /** Sends the rdd to the scheduler for scheduling. */ + /** Sends the rdd to the scheduler for scheduling and returns the job id. */ private def submit( rdd: RDD[_], partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener) { + listener: JobListener = listener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) + return jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -190,6 +198,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(TaskSetFailed(taskSet, message)) } + /** Sends JobCancelled to the DAG scheduler. */ + private def cancel(jobId: Int) { + runEvent(JobCancelled(jobId)) + } + test("zero split job") { val rdd = makeRdd(0, Nil) var numResults = 0 @@ -248,7 +261,15 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") - assert(failure.getMessage === "Job aborted: some failure") + assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assertDataStructuresEmpty + } + + test("trivial job cancellation") { + val rdd = makeRdd(1, Nil) + val jobId = submit(rdd, Array(0)) + cancel(jobId) + assert(failure.getMessage === s"Job $jobId cancelled") assertDataStructuresEmpty } @@ -323,6 +344,67 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("run shuffle with map stage failure") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val reduceRdd = makeRdd(2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + + // Fail the map stage. This should cause the entire job to fail. + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + + /** + * Makes sure that failures of stage used by multiple jobs are correctly handled. + * + * This test creates the following dependency graph: + * + * shuffleMapRdd1 shuffleMapRDD2 + * | \ | + * | \ | + * | \ | + * | \ | + * reduceRdd1 reduceRdd2 + * + * We start both shuffleMapRdds and then fail shuffleMapRdd1. As a result, the job listeners for + * reduceRdd1 and reduceRdd2 should both be informed that the job failed. shuffleMapRDD2 should + * also be cancelled, because it is only used by reduceRdd2 and reduceRdd2 cannot complete + * without shuffleMapRdd1. + */ + test("failure of stage used by two jobs") { + val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) + val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) + + val reduceRdd1 = makeRdd(2, List(shuffleDep1)) + val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + + // We need to make our own listeners for this test, since by default submit uses the same + // listener for all jobs, and here we want to capture the failure for each job separately. + class FailureRecordingJobListener() extends JobListener { + var failureMessage: String = _ + override def taskSucceeded(index: Int, result: Any) {} + override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + } + val listener1 = new FailureRecordingJobListener() + val listener2 = new FailureRecordingJobListener() + + submit(reduceRdd1, Array(0, 1), listener=listener1) + submit(reduceRdd2, Array(0, 1), listener=listener2) + + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + + assert(cancelledStages.contains(1)) + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + test("run trivial shuffle with out-of-band failure and retry") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 3bc054893bf2decdafa97a1e149e489ad154f066 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 8 Apr 2014 14:23:16 -0700 Subject: [PATCH 02/12] Remove extra semicolon in import statement and unused import in ApplicationMaster Small nit cleanup to remove extra semicolon and unused import in Yarn's stable ApplicationMaster (it bothers me every time I saw it) Author: Henry Saputra Closes #358 from hsaputra/nitcleanup_removesemicolon_import_applicationmaster and squashes the following commits: bffb685 [Henry Saputra] Remove extra semicolon in import statement and unused import in ApplicationMaster.scala --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 30735cbfdf26e..c8a4d2e647cbd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.yarn import java.io.IOException -import java.net.Socket import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} @@ -36,7 +35,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.hadoop.yarn.webapp.util.WebAppUtils; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil From a8d86b080ae26c96b078ba14dc60f3b528c07787 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 8 Apr 2014 14:30:24 -0700 Subject: [PATCH 03/12] SPARK-1348 binding Master, Worker, and App Web UI to all interfaces Author: Kan Zhang Closes #318 from kanzhang/SPARK-1348 and squashes the following commits: e625a5f [Kan Zhang] reverting the changes to startJettyServer() 7a8084e [Kan Zhang] SPARK-1348 binding Master, Worker, and App Web UI to all interfaces --- .../scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index bd75b2dfd0e07..01d9f52f4b7b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -59,7 +59,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def bind() { try { - serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index de76a5d5eb7bc..650f3da5ce3ff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -60,7 +60,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + serverInfo = Some(JettyUtils.startJettyServer("0.0.0.0", port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index ef1ad872c8ef7..f53df7fbedf39 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -80,7 +80,7 @@ private[spark] class SparkUI( /** Bind the HTTP server which backs this web interface */ def bind() { try { - serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => From e25b593447a2e0aab9e5066f755e41be9068ecdc Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 8 Apr 2014 14:40:20 -0700 Subject: [PATCH 04/12] SPARK-1445: compute-classpath should not print error if lib_managed not found This was added to the check for the assembly jar, forgot it for the datanucleus jars. Author: Aaron Davidson Closes #361 from aarondav/cc and squashes the following commits: 8facc16 [Aaron Davidson] SPARK-1445: compute-classpath should not print error if lib_managed not found --- bin/compute-classpath.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index be37102dc069a..2a2bb376fd71f 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -63,7 +63,7 @@ fi # built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark # assembly is built for Hive, before actually populating the CLASSPATH with the jars. # Note that this check order is faster (by up to half a second) in the case where Hive is not used. -num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ | grep "datanucleus-.*\\.jar" | wc -l) +num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ 2>/dev/null | grep "datanucleus-.*\\.jar" | wc -l) if [ $num_datanucleus_jars -gt 0 ]; then AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) From fac6085cd774a4dba73ad1618537ef1817b2bcf3 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Apr 2014 14:42:02 -0700 Subject: [PATCH 05/12] [SPARK-1397] Notify SparkListeners when stages fail or are cancelled. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit [I wanted to post this for folks to comment but it depends on (and thus includes the changes in) a currently outstanding PR, #305. You can look at just the second commit: https://github.com/kayousterhout/spark-1/commit/93f08baf731b9eaf5c9792a5373560526e2bccac to see just the changes relevant to this PR] Previously, when stages fail or get cancelled, the SparkListener is only notified indirectly through the SparkListenerJobEnd, where we sometimes pass in a single stage that failed. This worked before job cancellation, because jobs would only fail due to a single stage failure. However, with job cancellation, multiple running stages can fail when a job gets cancelled. Right now, this is not handled correctly, which results in stages that get stuck in the “Running Stages” window in the UI even though they’re dead. This PR changes the SparkListenerStageCompleted event to a SparkListenerStageEnded event, and uses this event to tell SparkListeners when stages fail in addition to when they complete successfully. This change is NOT publicly backward compatible for two reasons. First, it changes the SparkListener interface. We could alternately add a new event, SparkListenerStageFailed, and keep the existing SparkListenerStageCompleted. However, this is less consistent with the listener events for tasks / jobs ending, and will result in some code duplication for listeners (because failed and completed stages are handled in similar ways). Note that I haven’t finished updating the JSON code to correctly handle the new event because I’m waiting for feedback on whether this is a good or bad idea (hence the “WIP”). It is also not backwards compatible because it changes the publicly visible JobWaiter.jobFailed() method to no longer include a stage that caused the failure. I think this change should definitely stay, because with cancellation (as described above), a failure isn’t necessarily caused by a single stage. Author: Kay Ousterhout Closes #309 from kayousterhout/stage_cancellation and squashes the following commits: 5533ecd [Kay Ousterhout] Fixes in response to Mark's review 320c7c7 [Kay Ousterhout] Notify SparkListeners when stages fail or are cancelled. --- .../scala/org/apache/spark/FutureAction.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 121 +++++++++++------- .../apache/spark/scheduler/JobLogger.scala | 8 +- .../apache/spark/scheduler/JobResult.scala | 3 +- .../apache/spark/scheduler/JobWaiter.scala | 2 +- .../spark/scheduler/SparkListener.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 9 ++ .../spark/ui/jobs/JobProgressListener.scala | 23 +--- .../org/apache/spark/util/JsonProtocol.scala | 11 +- .../spark/scheduler/DAGSchedulerSuite.scala | 45 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 3 +- 11 files changed, 151 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index f2decd14ef6d9..2eec09cd1c795 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -141,7 +141,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: private def awaitResult(): Try[T] = { jobWaiter.awaitResult() match { case JobSucceeded => scala.util.Success(resultFunc) - case JobFailed(e: Exception, _) => scala.util.Failure(e) + case JobFailed(e: Exception) => scala.util.Failure(e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c96d7435a7ed4..c41d6d75a1d49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -342,22 +342,24 @@ class DAGScheduler( } /** - * Removes job and any stages that are not needed by any other job. Returns the set of ids for - * stages that were removed. The associated tasks for those stages need to be cancelled if we - * got here via job cancellation. + * Removes state for job and any stages that are not needed by any other job. Does not + * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. + * + * @param job The job whose state to cleanup. + * @param resultStage Specifies the result stage for the job; if set to None, this method + * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { - val registeredStages = jobIdToStageIds(jobId) - val independentStages = new HashSet[Int]() - if (registeredStages.isEmpty) { - logError("No stages registered for job " + jobId) + private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + val registeredStages = jobIdToStageIds.get(job.jobId) + if (registeredStages.isEmpty || registeredStages.get.isEmpty) { + logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { + stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, jobSet) => - if (!jobSet.contains(jobId)) { + if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" - .format(jobId, stageId)) + .format(job.jobId, stageId)) } else { def removeStage(stageId: Int) { // data structures based on Stage @@ -394,23 +396,28 @@ class DAGScheduler( .format(stageId, stageIdToStage.size)) } - jobSet -= jobId + jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage - independentStages += stageId removeStage(stageId) } } } } - independentStages.toSet - } + jobIdToStageIds -= job.jobId + jobIdToActiveJob -= job.jobId + activeJobs -= job - private def jobIdToStageIdsRemove(jobId: Int) { - if (!jobIdToStageIds.contains(jobId)) { - logDebug("Trying to remove unregistered job " + jobId) + if (resultStage.isEmpty) { + // Clean up result stages. + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob } else { - removeJobAndIndependentStages(jobId) - jobIdToStageIds -= jobId + resultStageToJob -= resultStage.get } } @@ -460,7 +467,7 @@ class DAGScheduler( val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { case JobSucceeded => {} - case JobFailed(exception: Exception, _) => + case JobFailed(exception: Exception) => logInfo("Failed to run " + callSite) throw exception } @@ -606,7 +613,16 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) + // Tell the listeners that all of the running stages have ended. Don't bother + // cancelling the stages because if the DAG scheduler is stopped, the entire application + // is in the process of getting stopped. + val stageFailedMessage = "Stage cancelled because SparkContext was shut down" + runningStages.foreach { stage => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } return true } @@ -676,7 +692,7 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, job.finalStage.id) + jobResult = JobFailed(e) job.listener.jobFailed(e) } finally { val s = job.finalStage @@ -826,11 +842,8 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - jobIdToActiveJob -= stage.jobId - activeJobs -= job - resultStageToJob -= stage markStageAsFinished(stage) - jobIdToStageIdsRemove(job.jobId) + cleanupStateForJobAndIndependentStages(job, Some(stage)) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) @@ -982,7 +995,7 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled") + failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None) } } @@ -999,7 +1012,8 @@ class DAGScheduler( stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", + Some(resultStage)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -1008,28 +1022,45 @@ class DAGScheduler( /** * Fails a job and all stages that are only used by that job, and cleans up relevant state. + * + * @param resultStage The result stage for the job, if known. Used to cleanup state for the job + * slightly more efficiently than when not specified. */ - private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, + resultStage: Option[Stage]) { val error = new SparkException(failureReason) job.listener.jobFailed(error) - // Cancel all tasks in independent stages. - val independentStages = removeJobAndIndependentStages(job.jobId) - independentStages.foreach(taskScheduler.cancelTasks) - - // Clean up remaining state we store for the job. - jobIdToActiveJob -= job.jobId - activeJobs -= job - jobIdToStageIds -= job.jobId - val resultStagesForJob = resultStageToJob.keySet.filter( - stage => resultStageToJob(stage).jobId == job.jobId) - if (resultStagesForJob.size != 1) { - logWarning( - s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + // Cancel all independent, running stages. + val stages = jobIdToStageIds(job.jobId) + if (stages.isEmpty) { + logError("No stages registered for job " + job.jobId) } - resultStageToJob --= resultStagesForJob + stages.foreach { stageId => + val jobsForStage = stageIdToJobIds.get(stageId) + if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { + logError( + "Job %d not registered for stage %d even though that stage was registered for the job" + .format(job.jobId, stageId)) + } else if (jobsForStage.get.size == 1) { + if (!stageIdToStage.contains(stageId)) { + logError("Missing Stage for stage with id $stageId") + } else { + // This is the only job that uses this stage, so fail the stage if it is running. + val stage = stageIdToStage(stageId) + if (runningStages.contains(stage)) { + taskScheduler.cancelTasks(stageId) + val stageInfo = stageToInfos(stage) + stageInfo.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + } + } + } + } + + cleanupStateForJobAndIndependentStages(job, resultStage) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 5cecf9416b32c..7c5053998f1d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -191,7 +191,11 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageId = stageCompleted.stageInfo.stageId - stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) + if (stageCompleted.stageInfo.failureReason.isEmpty) { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=COMPLETED") + } else { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=FAILED") + } } /** @@ -227,7 +231,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception, _) => + case JobFailed(exception) => info += " STATUS=FAILED REASON=" exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 3cf4e3077e4a4..047bd27056120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -24,5 +24,4 @@ private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -// A failed stage ID of -1 means there is not a particular stage that caused the failure -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult +private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 8007b5418741e..e9bfee2248e5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, -1) + jobResult = JobFailed(exception) this.notifyAll() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d4eb0ac88d8e8..d42e67742a4f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -71,7 +71,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent */ trait SparkListener { /** - * Called when a stage is completed, with information on the completed stage + * Called when a stage completes successfully or fails, with information on the completed stage. */ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8115a7ed7896d..eec409b182ac6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -26,8 +26,17 @@ private[spark] class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None + /** Time when all tasks in the stage completed or when the stage was cancelled. */ var completionTime: Option[Long] = None + /** If the stage failed, the reason why. */ + var failureReason: Option[String] = None + var emittedTaskSizeWarning = false + + def stageFailed(reason: String) { + failureReason = Some(reason) + completionTime = Some(System.currentTimeMillis) + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 048f671c8788f..5167e20ea3d7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -74,8 +74,13 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage poolToActiveStages(stageIdToPool(stageId)).remove(stageId) activeStages.remove(stageId) - completedStages += stage - trimIfNecessary(completedStages) + if (stage.failureReason.isEmpty) { + completedStages += stage + trimIfNecessary(completedStages) + } else { + failedStages += stage + trimIfNecessary(failedStages) + } } /** If stages is too large, remove and garbage collect old stages */ @@ -215,20 +220,6 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd.jobResult match { - case JobFailed(_, stageId) => - activeStages.get(stageId).foreach { s => - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage - activeStages.remove(s.stageId) - poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) - failedStages += s - trimIfNecessary(failedStages) - } - case _ => - } - } - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { val schedulingModeName = diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2155a8888c85c..19654892bf661 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -166,12 +166,14 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoToJson(stageInfo.rddInfo) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) + val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ + ("Failure Reason" -> failureReason) ~ ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) } @@ -259,9 +261,7 @@ private[spark] object JsonProtocol { val json = jobResult match { case JobSucceeded => Utils.emptyJson case jobFailed: JobFailed => - val exception = exceptionToJson(jobFailed.exception) - ("Exception" -> exception) ~ - ("Failed Stage ID" -> jobFailed.failedStageId) + JObject("Exception" -> exceptionToJson(jobFailed.exception)) } ("Result" -> result) ~ json } @@ -442,11 +442,13 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoFromJson(json \ "RDD Info") val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) + val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime + stageInfo.failureReason = failureReason stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning stageInfo } @@ -561,8 +563,7 @@ private[spark] object JsonProtocol { case `jobSucceeded` => JobSucceeded case `jobFailed` => val exception = exceptionFromJson(json \ "Exception") - val failedStageId = (json \ "Failed Stage ID").extract[Int] - new JobFailed(exception, failedStageId) + new JobFailed(exception) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2e3026bffba2f..a74724d785ad3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -64,6 +64,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def defaultParallelism() = 2 } + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + val sparkListener = new SparkListener() { + val successfulStages = new HashSet[Int]() + val failedStages = new HashSet[Int]() + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + val stageInfo = stageCompleted.stageInfo + if (stageInfo.failureReason.isEmpty) { + successfulStages += stageInfo.stageId + } else { + failedStages += stageInfo.stageId + } + } + } + var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null @@ -89,13 +104,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** The list of results that DAGScheduler has collected. */ val results = new HashMap[Int, Any]() var failure: Exception = _ - val listener = new JobListener() { + val jobListener = new JobListener() { override def taskSucceeded(index: Int, result: Any) = results.put(index, result) override def jobFailed(exception: Exception) = { failure = exception } } before { sc = new SparkContext("local", "DAGSchedulerSuite") + sparkListener.successfulStages.clear() + sparkListener.failedStages.clear() + sc.addSparkListener(sparkListener) taskSets.clear() cancelledStages.clear() cacheLocations.clear() @@ -187,7 +205,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener): Int = { + listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) return jobId @@ -231,7 +249,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) + runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, jobListener)) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } @@ -262,6 +280,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -270,6 +291,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -354,6 +378,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val stageFailureMessage = "Exception failure in map stage" failed(taskSets(0), stageFailureMessage) assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + + // Listener bus should get told about the map stage failing, but not the reduce stage + // (since the reduce stage hasn't been started yet). + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty } @@ -400,6 +431,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont failed(taskSets(0), stageFailureMessage) assert(cancelledStages.contains(1)) + + // Make sure the listeners got told about both failed stages. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.successfulStages.isEmpty) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.contains(3)) + assert(sparkListener.failedStages.size === 2) + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assertDataStructuresEmpty diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 7bab7da8fed68..0342a8aff3c28 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -89,7 +89,7 @@ class JsonProtocolSuite extends FunSuite { // JobResult val exception = new Exception("Out of Memory! Please restock film.") exception.setStackTrace(stackTrace) - val jobFailed = JobFailed(exception, 2) + val jobFailed = JobFailed(exception) testJobResult(JobSucceeded) testJobResult(jobFailed) @@ -294,7 +294,6 @@ class JsonProtocolSuite extends FunSuite { (result1, result2) match { case (JobSucceeded, JobSucceeded) => case (r1: JobFailed, r2: JobFailed) => - assert(r1.failedStageId === r2.failedStageId) assertEquals(r1.exception, r2.exception) case _ => fail("Job results don't match in types!") } From 12c077d5aa0b76a808a55db625c9677a52bd43f9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Tue, 8 Apr 2014 16:19:22 -0700 Subject: [PATCH 06/12] SPARK-1433: Upgrade Mesos dependency to 0.17.0 Mesos 0.13.0 was released 6 months ago. Upgrade Mesos dependency to 0.17.0 Author: Sandeep Closes #355 from techaddict/mesos_update and squashes the following commits: f1abeee [Sandeep] SPARK-1433: Upgrade Mesos dependency to 0.17.0 Mesos 0.13.0 was released 6 months ago. Upgrade Mesos dependency to 0.17.0 --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++++-- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/_config.yml | 2 +- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..c478e685641d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,10 +194,12 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(offer.getId, Collections.singletonList(task), filters) + d.launchTasks(Collections.singletonList(offer.getId), + Collections.singletonList(task), + filters) } else { // Filter it out - d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) + d.declineOffer(offer.getId, filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..f878ae338fc95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + d.launchTasks(Collections.singletonList(offers(i).getId), mesosTasks(i), filters) } } } finally { diff --git a/docs/_config.yml b/docs/_config.yml index d585b8c5ea763..bd5ed6c9220d2 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.13.0 +MESOS_VERSION: 0.17.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index c03bb35c99442..11511bcb9da52 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 2.10.4 2.10 - 0.13.0 + 0.17.0 org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -848,7 +848,7 @@ - + hadoop-provided @@ -893,6 +893,6 @@ - + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6b8740d9f21a1..08667aac2cd2d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -332,7 +332,7 @@ object SparkBuild extends Build { "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.17.0", "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From ce8ec5456169682f27f846e7b8d51e6c4bcf75e3 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 18:15:52 -0700 Subject: [PATCH 07/12] Spark 1271: Co-Group and Group-By should pass Iterable[X] Author: Holden Karau Closes #242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits: f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator 77048f8 [Holden Karau] Fix merge up to master d3fe909 [Holden Karau] use toSeq instead 7a092a3 [Holden Karau] switch resultitr to resultiterable eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables c5075aa [Holden Karau] If guava 14 had iterables 2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API 11e730c [Holden Karau] Fix streaming tests 66b583d [Holden Karau] Fix the core test suite to compile 4ed579b [Holden Karau] Refactor from iterator to iterable d052c07 [Holden Karau] Python tests now pass with iterator pandas 3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work" cd1e81c [Holden Karau] Try and make pickling list iterators work c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well 88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming a5ee714 [Holden Karau] oops, was checking wrong iterator e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming ec8cc3e [Holden Karau] Fix test issues\! 4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD" ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas" b692868 [Holden Karau] Revert 7e533f7 [Holden Karau] Fix the bug 8a5153a [Holden Karau] Revert me, but we have some stuff to debug b4e86a9 [Holden Karau] Add a join based on the problem in SVD c4510e2 [Holden Karau] Revert this but for now put things in list pandas b4e0b1d [Holden Karau] Fix style issues 71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness. b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work 37888ec [Holden Karau] core/tests now pass 249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes 6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy" fe992fe [Holden Karau] hmmm try and fix up basic operation suite 172705c [Holden Karau] Fix Java API suite caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy 88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator 4991af6 [Holden Karau] Fix some tests be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after 687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures --- .../scala/org/apache/spark/bagel/Bagel.scala | 20 ++++--- .../apache/spark/api/java/JavaPairRDD.scala | 36 ++++++------ .../apache/spark/api/java/JavaRDDLike.scala | 6 +- .../apache/spark/rdd/PairRDDFunctions.scala | 39 +++++++------ .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../java/org/apache/spark/JavaAPISuite.java | 20 ++++--- .../scala/org/apache/spark/FailureSuite.scala | 4 +- .../org/apache/spark/PipedRDDSuite.scala | 2 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 12 ++-- .../ExternalAppendOnlyMapSuite.scala | 4 +- .../apache/spark/examples/JavaPageRank.java | 21 ++++--- .../bagel/WikipediaPageRankStandalone.scala | 14 +++-- .../java/org/apache/spark/Java8APISuite.java | 11 ++-- .../org/apache/spark/mllib/linalg/SVD.scala | 6 +- .../spark/mllib/recommendation/ALS.scala | 4 +- .../org/apache/spark/mllib/util/LAUtils.scala | 6 +- python/pyspark/join.py | 5 +- python/pyspark/rdd.py | 10 ++-- python/pyspark/resultiterable.py | 33 +++++++++++ .../streaming/api/java/JavaPairDStream.scala | 42 +++++++------- .../dstream/PairDStreamFunctions.scala | 29 +++++----- .../streaming/dstream/StateDStream.scala | 13 +++-- .../apache/spark/streaming/JavaAPISuite.java | 58 ++++++++++++++++--- .../streaming/BasicOperationsSuite.scala | 4 +- 24 files changed, 252 insertions(+), 153 deletions(-) create mode 100644 python/pyspark/resultiterable.py diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70c7474a936dc..70a99b33d753c 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -220,20 +220,23 @@ object Bagel extends Logging { */ private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, - grouped: RDD[(K, (Seq[C], Seq[V]))], + grouped: RDD[(K, (Iterable[C], Iterable[V]))], compute: (V, Option[C]) => (V, Array[M]), storageLevel: StorageLevel ): (RDD[(K, (V, Array[M]))], Int, Int) = { var numMsgs = sc.accumulator(0) var numActiveVerts = sc.accumulator(0) - val processed = grouped.flatMapValues { - case (_, vs) if vs.size == 0 => None - case (c, vs) => + val processed = grouped.mapValues(x => (x._1.iterator, x._2.iterator)) + .flatMapValues { + case (_, vs) if !vs.hasNext => None + case (c, vs) => { val (newVert, newMsgs) = - compute(vs(0), c match { - case Seq(comb) => Some(comb) - case Seq() => None - }) + compute(vs.next, + c.hasNext match { + case true => Some(c.next) + case false => None + } + ) numMsgs += newMsgs.size if (newVert.active) { @@ -241,6 +244,7 @@ object Bagel extends Logging { } Some((newVert, newMsgs)) + } }.persist(storageLevel) // Force evaluation of processed RDD for accurate performance measurements diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 9596dbaf75488..e6c5d85917678 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -250,14 +251,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JList[V]] = + def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = + def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** @@ -367,7 +368,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): JavaPairRDD[K, JList[V]] = + def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) /** @@ -462,7 +463,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) /** @@ -470,14 +471,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], - partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + partitioner: Partitioner): JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) /** @@ -485,7 +486,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) /** @@ -493,7 +494,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** @@ -501,16 +502,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) /** @@ -695,21 +696,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] - def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Seq[T])]): RDD[(K, JList[T])] = { - rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList) + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterable[T])]): RDD[(K, JIterable[T])] = { + rddToPairRDDFunctions(rdd).mapValues(asJavaIterable) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( - rdd: RDD[(K, (Seq[V], Seq[W]))]): RDD[(K, (JList[V], JList[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + rdd: RDD[(K, (Iterable[V], Iterable[W]))]): RDD[(K, (JIterable[V], JIterable[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2))) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))]): RDD[(K, (JList[V], JList[W1], JList[W2]))] = { + rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]) + : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2]))] = { rddToPairRDDFunctions(rdd) - .mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) + .mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3))) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 6e8ec8e0c7629..ae577b500ccb4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.util.{Comparator, List => JList, Iterator => JIterator} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -204,7 +204,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) @@ -214,7 +214,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 14386ff5b9127..a92a84b5342d1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. @@ -270,14 +270,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Seq[V])]] + bufs.mapValues(_.toIterable) } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) } @@ -298,7 +298,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + for (v <- vs; w <- ws) yield (v, w) } } @@ -311,9 +311,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) + vs.map(v => (v, None)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + for (v <- vs; w <- ws) yield (v, Some(w)) } } } @@ -328,9 +328,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) + ws.map(w => (None, w)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + for (v <- vs; w <- ws) yield (Some(v), w) } } } @@ -358,7 +358,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): RDD[(K, Seq[V])] = { + def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) } @@ -453,7 +453,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Iterable[V], Iterable[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -468,13 +469,15 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]]) } } @@ -482,7 +485,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -491,7 +494,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -499,7 +502,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -508,18 +511,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } 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 bf3c57ad41eb2..74fa2a4fcd401 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -438,20 +438,20 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 762405be2a8f9..ab2fdac553349 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,10 +18,12 @@ package org.apache.spark; import java.io.*; +import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; @@ -197,7 +199,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size()); + Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -209,15 +211,15 @@ public Boolean call(Integer x) { return x % 2 == 0; } }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -232,9 +234,9 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, List>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString()); - Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString()); + JavaPairRDD, Iterable>> cogrouped = categories.cogroup(prices); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index f3fb64d87a2fd..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,7 +72,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - (k, v(0) * v(0)) + (k, v.head * v.head) }.collect() FailureSuiteState.synchronized { assert(FailureSuiteState.tasksRun === 4) @@ -137,5 +137,3 @@ class FailureSuite extends FunSuite with LocalSparkContext { // TODO: Need to add tests with shuffle fetch failures. } - - diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 627e9b5cd9060..867b28cc0d971 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -85,7 +85,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { (f: String => Unit) => { bl.value.map(f(_)); f("\u0001") }, - (i: Tuple2[String, Seq[String]], f: String => Unit) => { + (i: Tuple2[String, Iterable[String]], f: String => Unit) => { for (e <- i._2) { f(e + "_") } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index f9e994b13dfbc..8f3e6bd21b752 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -225,11 +225,12 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) + val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'))), + (2, (List(1), List('y', 'z'))), + (3, (List(1), List())), + (4, (List(), List('w'))) )) } @@ -447,4 +448,3 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { super.getRecordWriter(p1) } } - diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index fce1184d46364..cdebefb67510c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -174,9 +174,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey - val result2 = rdd.groupByKey().collect() + val result2 = rdd.groupByKey().collect().map(x => (x._1, x._2.toList)).toSet assert(result2.toSet == Set[(Int, Seq[Int])] - ((0, ArrayBuffer[Int](1, 1, 1, 1, 1)), (1, ArrayBuffer[Int](1, 1, 1, 1, 1)))) + ((0, List[Int](1, 1, 1, 1, 1)), (1, List[Int](1, 1, 1, 1, 1)))) } test("simple cogroup") { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index eb70fb547564c..8513ba07e7705 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,7 +17,10 @@ package org.apache.spark.examples; + import scala.Tuple2; + +import com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -26,8 +29,9 @@ import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; -import java.util.List; import java.util.ArrayList; +import java.util.List; +import java.util.Iterator; import java.util.regex.Pattern; /** @@ -66,7 +70,7 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.mapToPair(new PairFunction() { + JavaPairRDD> links = lines.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { String[] parts = SPACES.split(s); @@ -75,9 +79,9 @@ public Tuple2 call(String s) { }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = links.mapValues(new Function, Double>() { + JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) { + public Double call(Iterable rs) { return 1.0; } }); @@ -86,12 +90,13 @@ public Double call(List rs) { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { + .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override - public Iterable> call(Tuple2, Double> s) { + public Iterable> call(Tuple2, Double> s) { + int urlCount = Iterables.size(s._1); List> results = new ArrayList>(); - for (String n : s._1()) { - results.add(new Tuple2(n, s._2() / s._1().size())); + for (String n : s._1) { + results.add(new Tuple2(n, s._2() / urlCount)); } return results; } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 27afa6b642758..7aac6a13597e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -115,12 +115,16 @@ object WikipediaPageRankStandalone { var ranks = links.mapValues { edges => defaultRank } for (i <- 1 to numIterations) { val contribs = links.groupWith(ranks).flatMap { - case (id, (linksWrapper, rankWrapper)) => - if (linksWrapper.length > 0) { - if (rankWrapper.length > 0) { - linksWrapper(0).map(dest => (dest, rankWrapper(0) / linksWrapper(0).size)) + case (id, (linksWrapperIterable, rankWrapperIterable)) => + val linksWrapper = linksWrapperIterable.iterator + val rankWrapper = rankWrapperIterable.iterator + if (linksWrapper.hasNext) { + val linksWrapperHead = linksWrapper.next + if (rankWrapper.hasNext) { + val rankWrapperHead = rankWrapper.next + linksWrapperHead.map(dest => (dest, rankWrapperHead / linksWrapperHead.size)) } else { - linksWrapper(0).map(dest => (dest, defaultRank / linksWrapper(0).size)) + linksWrapperHead.map(dest => (dest, defaultRank / linksWrapperHead.size)) } } else { Array[(String, Double)]() diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index f67251217ed4a..7eb8b45fc3cf0 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,6 +23,7 @@ import scala.Tuple2; +import com.google.common.collections.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; @@ -85,15 +86,15 @@ public void foreach() { public void groupBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Function isOdd = x -> x % 2 == 0; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @Test diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 3e7cc648d1d37..0d97b7d92f155 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -69,11 +69,11 @@ class SVD { /** * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T + * Returns (U, S, V) such that A = USV^T * U is a row-by-row dense matrix * S is a simple double array of singular values * V is a 2d array matrix - * See [[denseSVD]] for more documentation + * See [[denseSVD]] for more documentation */ def compute(matrix: RDD[Array[Double]]): (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { @@ -393,5 +393,3 @@ object SVD { System.exit(0) } } - - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 0cc9f48769f83..3124fac326d22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -421,12 +421,12 @@ class ALS private ( * Compute the new feature vectors for a block of the users matrix given the list of factors * it received from each product and its InLinkBlock. */ - private def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + private def updateBlock(messages: Iterable[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) : Array[Array[Double]] = { // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] + val blockFactors = messages.toSeq.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] val numBlocks = blockFactors.length val numUsers = inLinkBlock.elementIds.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala index afe081295bfae..87aac347579c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -38,8 +38,10 @@ object LAUtils { case (i, cols) => val rowArray = Array.ofDim[Double](n) var j = 0 - while (j < cols.size) { - rowArray(cols(j)._1) = cols(j)._2 + val colsItr = cols.iterator + while (colsItr.hasNext) { + val element = colsItr.next + rowArray(element._1) = element._2 j += 1 } MatrixRow(i, rowArray) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 5f4294fb1b777..6f94d26ef86a9 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -31,11 +31,12 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ +from pyspark.resultiterable import ResultIterable def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x : dispatch(x.__iter__())) def python_join(rdd, other, numPartitions): @@ -88,5 +89,5 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return (vbuf, wbuf) + return (ResultIterable(vbuf), ResultIterable(wbuf)) return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index fb27863e07f55..91fc7e637e2c6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -38,6 +38,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler from pyspark.storagelevel import StorageLevel +from pyspark.resultiterable import ResultIterable from py4j.java_collections import ListConverter, MapConverter @@ -1118,7 +1119,7 @@ def groupByKey(self, numPartitions=None): Hash-partitions the resulting RDD with into numPartitions partitions. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().collect()) + >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) [('a', [1, 1]), ('b', [1])] """ @@ -1133,7 +1134,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions) + numPartitions).mapValues(lambda x: ResultIterable(x)) # TODO: add tests def flatMapValues(self, f): @@ -1180,7 +1181,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup(self, other, numPartitions) @@ -1217,7 +1218,7 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] """ return self.map(lambda x: (f(x), x)) @@ -1317,7 +1318,6 @@ def getStorageLevel(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. - class PipelinedRDD(RDD): """ Pipelined maps: diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py new file mode 100644 index 0000000000000..7f418f8d2e29a --- /dev/null +++ b/python/pyspark/resultiterable.py @@ -0,0 +1,33 @@ +# +# 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. +# + +__all__ = ["ResultIterable"] + +import collections + +class ResultIterable(collections.Iterable): + """ + A special result iterable. This is used because the standard iterator can not be pickled + """ + def __init__(self, data): + self.data = data + self.index = 0 + self.maxindex = len(data) + def __iter__(self): + return iter(self.data) + def __len__(self): + return len(self.data) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index ac451d1913aaa..2ac943d7bf781 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong} +import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConversions._ @@ -115,15 +115,15 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): JavaPairDStream[K, JList[V]] = - dstream.groupByKey().mapValues(seqAsJavaList _) + def groupByKey(): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey().mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _) + def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(numPartitions).mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. @@ -131,8 +131,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner * is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(partitioner).mapValues(seqAsJavaList _) + def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(partitioner).mapValues(asJavaIterable _) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -196,8 +196,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration).mapValues(seqAsJavaList _) + def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterable _) } /** @@ -211,8 +211,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(seqAsJavaList _) + : JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterable _) } /** @@ -227,9 +227,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - :JavaPairDStream[K, JList[V]] = { + :JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterable _) } /** @@ -247,9 +247,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ):JavaPairDStream[K, JList[V]] = { + ):JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterable _) } /** @@ -518,9 +518,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = { + def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + dstream.cogroup(other.dstream).mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -530,10 +530,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], numPartitions: Int - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, numPartitions) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -543,10 +543,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], partitioner: Partitioner - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, partitioner) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 2473496949360..354bc132dcdc0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -51,7 +51,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): DStream[(K, Seq[V])] = { + def groupByKey(): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner()) } @@ -59,7 +59,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner(numPartitions)) } @@ -67,12 +67,12 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): DStream[(K, Iterable[V])] = { val createCombiner = (v: V) => ArrayBuffer[V](v) val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v) val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2) combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -126,7 +126,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = { + def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner()) } @@ -140,7 +140,8 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval */ - def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = + def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) + : DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner()) } @@ -161,7 +162,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, numPartitions: Int - ): DStream[(K, Seq[V])] = { + ): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions)) } @@ -180,14 +181,14 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ): DStream[(K, Seq[V])] = { - val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v - val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v + ): DStream[(K, Iterable[V])] = { + val createCombiner = (v: Iterable[V]) => new ArrayBuffer[V] ++= v + val mergeValue = (buf: ArrayBuffer[V], v: Iterable[V]) => buf ++= v val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2 self.groupByKey(partitioner) .window(windowDuration, slideDuration) .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -438,7 +439,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner()) } @@ -447,7 +448,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int) - : DStream[(K, (Seq[V], Seq[W]))] = { + : DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(numPartitions)) } @@ -458,7 +459,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner - ): DStream[(K, (Seq[V], Seq[W]))] = { + ): DStream[(K, (Iterable[V], Iterable[W]))] = { self.transformWith( other, (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.cogroup(rdd2, partitioner) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 5f7d3ba26c656..7e22268767de7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -56,9 +56,14 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the cogrouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, (Seq[V], Seq[S]))]) => { + val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))]) => { val i = iterator.map(t => { - (t._1, t._2._1, t._2._2.headOption) + val itr = t._2._2.iterator + val headOption = itr.hasNext match { + case true => Some(itr.next()) + case false => None + } + (t._1, t._2._1.toSeq, headOption) }) updateFuncLocal(i) } @@ -90,8 +95,8 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, Seq[V])]) => { - updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2, None))) + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { + updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2.toSeq, None))) } val groupedRDD = parentRDD.groupByKey(partitioner) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index e93bf18b6d0b9..13fa64894b773 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -23,6 +23,7 @@ import org.junit.Test; import java.io.*; import java.util.*; +import java.lang.Iterable; import com.google.common.base.Optional; import com.google.common.collect.Lists; @@ -45,6 +46,18 @@ // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { + public void equalIterator(Iterator a, Iterator b) { + while (a.hasNext() && b.hasNext()) { + Assert.assertEquals(a.next(), b.next()); + } + Assert.assertEquals(a.hasNext(), b.hasNext()); + } + + public void equalIterable(Iterable a, Iterable b) { + equalIterator(a.iterator(), b.iterator()); + } + + @SuppressWarnings("unchecked") @Test public void testCount() { @@ -1016,11 +1029,24 @@ public void testPairGroupByKey() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> grouped = pairStream.groupByKey(); + JavaPairDStream> grouped = pairStream.groupByKey(); JavaTestUtils.attachTestOutputStream(grouped); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator>>> resultItr = result.iterator(); + Iterator>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator>> resultElements = resultItr.next().iterator(); + Iterator>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2> resultElement = resultElements.next(); + Tuple2> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2(), resultElement._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") @@ -1128,7 +1154,7 @@ public void testGroupByKeyAndWindow() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> groupWindowed = + JavaPairDStream> groupWindowed = pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1471,11 +1497,25 @@ public void testCoGroup() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream, List>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List, List>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator, Iterable>>>> resultItr = result.iterator(); + Iterator, List>>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator, Iterable>>> resultElements = resultItr.next().iterator(); + Iterator, List>>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2, Iterable>> resultElement = resultElements.next(); + Tuple2, List>> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2()._1(), resultElement._2()._1()); + equalIterable(expectedElement._2()._2(), resultElement._2()._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index bb73dbf29b649..8aec27e39478a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -117,7 +117,7 @@ class BasicOperationsSuite extends TestSuiteBase { test("groupByKey") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), - (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(), + (s: DStream[String]) => s.map(x => (x, 1)).groupByKey().mapValues(_.toSeq), Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ), true ) @@ -251,7 +251,7 @@ class BasicOperationsSuite extends TestSuiteBase { Seq( ) ) val operation = (s1: DStream[String], s2: DStream[String]) => { - s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))) + s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))).mapValues(x => (x._1.toSeq, x._2.toSeq)) } testOperation(inputData1, inputData2, operation, outputData, true) } From b9e0c937dfa1ca93b63d0b39d5f156b16c2fdc0a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 20:37:01 -0700 Subject: [PATCH 08/12] [SPARK-1434] [MLLIB] change labelParser from anonymous function to trait This is a patch to address @mateiz 's comment in https://github.com/apache/spark/pull/245 MLUtils#loadLibSVMData uses an anonymous function for the label parser. Java users won't like it. So I make a trait for LabelParser and provide two implementations: binary and multiclass. Author: Xiangrui Meng Closes #345 from mengxr/label-parser and squashes the following commits: ac44409 [Xiangrui Meng] use singleton objects for label parsers 3b1a7c6 [Xiangrui Meng] add tests for label parsers c2e571c [Xiangrui Meng] rename LabelParser.apply to LabelParser.parse use extends for singleton 11c94e0 [Xiangrui Meng] add return types 7f8eb36 [Xiangrui Meng] change labelParser from annoymous function to trait --- .../spark/mllib/util/LabelParsers.scala | 49 +++++++++++++++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 28 ++--------- .../spark/mllib/util/LabelParsersSuite.scala | 41 ++++++++++++++++ .../spark/mllib/util/MLUtilsSuite.scala | 4 +- 4 files changed, 97 insertions(+), 25 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala new file mode 100644 index 0000000000000..f7966d3ebb613 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala @@ -0,0 +1,49 @@ +/* + * 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.mllib.util + +/** Trait for label parsers. */ +trait LabelParser extends Serializable { + /** Parses a string label into a double label. */ + def parse(labelString: String): Double +} + +/** + * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, + * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. + */ +object BinaryLabelParser extends LabelParser { + /** Gets the default instance of BinaryLabelParser. */ + def getInstance(): LabelParser = this + + /** + * Parses the input label into positive (1.0) if the value is greater than 0.5, + * or negative (0.0) otherwise. + */ + override def parse(labelString: String): Double = if (labelString.toDouble > 0.5) 1.0 else 0.0 +} + +/** + * Label parser for multiclass labels, which converts the input label to double. + */ +object MulticlassLabelParser extends LabelParser { + /** Gets the default instance of MulticlassLabelParser. */ + def getInstance(): LabelParser = this + + override def parse(labelString: String): Double = labelString.toDouble +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index cb85e433bfc73..83d1bd3fd57fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -38,17 +38,6 @@ object MLUtils { eps } - /** - * Multiclass label parser, which parses a string into double. - */ - val multiclassLabelParser: String => Double = _.toDouble - - /** - * Binary label parser, which outputs 1.0 (positive) if the value is greater than 0.5, - * or 0.0 (negative) otherwise. - */ - val binaryLabelParser: String => Double = label => if (label.toDouble > 0.5) 1.0 else 0.0 - /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. @@ -69,7 +58,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double, + labelParser: LabelParser, numFeatures: Int, minSplits: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minSplits) @@ -89,7 +78,7 @@ object MLUtils { }.reduce(math.max) } parsed.map { items => - val label = labelParser(items.head) + val label = labelParser.parse(items.head) val (indices, values) = items.tail.map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 @@ -107,14 +96,7 @@ object MLUtils { * with number of features determined automatically and the default number of partitions. */ def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMData(sc, path, binaryLabelParser, -1, sc.defaultMinSplits) - - /** - * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with number of features specified explicitly and the default number of partitions. - */ - def loadLibSVMData(sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMData(sc, path, binaryLabelParser, numFeatures, sc.defaultMinSplits) + loadLibSVMData(sc, path, BinaryLabelParser, -1, sc.defaultMinSplits) /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], @@ -124,7 +106,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double): RDD[LabeledPoint] = + labelParser: LabelParser): RDD[LabeledPoint] = loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinSplits) /** @@ -135,7 +117,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double, + labelParser: LabelParser, numFeatures: Int): RDD[LabeledPoint] = loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala new file mode 100644 index 0000000000000..ac85677f2f014 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala @@ -0,0 +1,41 @@ +/* + * 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.mllib.util + +import org.scalatest.FunSuite + +class LabelParsersSuite extends FunSuite { + test("binary label parser") { + for (parser <- Seq(BinaryLabelParser, BinaryLabelParser.getInstance())) { + assert(parser.parse("+1") === 1.0) + assert(parser.parse("1") === 1.0) + assert(parser.parse("0") === 0.0) + assert(parser.parse("-1") === 0.0) + } + } + + test("multiclass label parser") { + for (parser <- Seq(MulticlassLabelParser, MulticlassLabelParser.getInstance())) { + assert(parser.parse("0") == 0.0) + assert(parser.parse("+1") === 1.0) + assert(parser.parse("1") === 1.0) + assert(parser.parse("2") === 2.0) + assert(parser.parse("3") === 3.0) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 27d41c7869aa0..e451c350b8d88 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -80,7 +80,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, 6).collect() + val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, BinaryLabelParser, 6).collect() val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { @@ -93,7 +93,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MLUtils.multiclassLabelParser).collect() + val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MulticlassLabelParser).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) assert(multiclassPoints(1).label === -1.0) From fa0524fd02eedd0bbf1edc750dc3997a86ea25f5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 22:29:21 -0700 Subject: [PATCH 09/12] Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers. --- .rat-excludes | 2 +- .../scala/org/apache/spark/TestUtils.scala | 20 +++-- .../org/apache/spark/executor/Executor.scala | 17 +++-- .../executor/ExecutorURLClassLoader.scala | 45 ++++++++++- .../apache/spark/util/ParentClassLoader.scala | 32 ++++++++ .../ExecutorURLClassLoaderSuite.scala | 67 ++++++++++++++++ docs/configuration.md | 9 +++ project/SparkBuild.scala | 1 + .../spark/repl/ExecutorClassLoader.scala | 39 +++++++--- .../spark/repl/ExecutorClassLoaderSuite.scala | 76 +++++++++++++++++++ 10 files changed, 287 insertions(+), 21 deletions(-) rename core/src/{test => main}/scala/org/apache/spark/TestUtils.scala (84%) create mode 100644 core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala create mode 100644 core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala create mode 100644 repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala diff --git a/.rat-excludes b/.rat-excludes index 85bfad60fcadc..a2b5665a0be26 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,4 +39,4 @@ work .*\.q golden test.out/* -.*iml +.*iml \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala similarity index 84% rename from core/src/test/scala/org/apache/spark/TestUtils.scala rename to core/src/main/scala/org/apache/spark/TestUtils.scala index 1611d09652d40..4597595a838e3 100644 --- a/core/src/test/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -26,7 +26,14 @@ import scala.collection.JavaConversions._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import com.google.common.io.Files -object TestUtils { +/** + * Utilities for tests. Included in main codebase since it's used by multiple + * projects. + * + * TODO: See if we can move this to the test codebase by specifying + * test dependencies between projects. + */ +private[spark] object TestUtils { /** * Create a jar that defines classes with the given names. @@ -34,13 +41,14 @@ object TestUtils { * Note: if this is used during class loader tests, class names should be unique * in order to avoid interference between tests. */ - def createJarWithClasses(classNames: Seq[String]): URL = { + def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() - val files = for (name <- classNames) yield createCompiledClass(name, tempDir) + val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) } + /** * Create a jar file that contains this set of files. All files will be located at the root * of the jar. @@ -80,9 +88,11 @@ object TestUtils { } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ - def createCompiledClass(className: String, destDir: File): File = { + def createCompiledClass(className: String, destDir: File, value: String = ""): File = { val compiler = ToolProvider.getSystemJavaCompiler - val sourceFile = new JavaSourceFromString(className, s"public class $className {}") + val sourceFile = new JavaSourceFromString(className, + "public class " + className + " { @Override public String toString() { " + + "return \"" + value + "\";}}") // Calling this outputs a class file in pwd. It's easier to just rename the file than // build a custom FileManager that controls the output location. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index aecb069e4202b..c12bd922d40e4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -291,15 +291,19 @@ private[spark] class Executor( * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(): ExecutorURLClassLoader = { - val loader = Thread.currentThread().getContextClassLoader + private def createClassLoader(): MutableURLClassLoader = { + val loader = this.getClass.getClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. val urls = currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL }.toArray - new ExecutorURLClassLoader(urls, loader) + val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) + userClassPathFirst match { + case true => new ChildExecutorURLClassLoader(urls, loader) + case false => new ExecutorURLClassLoader(urls, loader) + } } /** @@ -310,11 +314,14 @@ private[spark] class Executor( val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) + val userClassPathFirst: java.lang.Boolean = + conf.getBoolean("spark.files.userClassPathFirst", false) try { val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) - constructor.newInstance(classUri, parent) + val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader], + classOf[Boolean]) + constructor.newInstance(classUri, parent, userClassPathFirst) } catch { case _: ClassNotFoundException => logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index f9bfe8ed2f5ba..208e77073fd03 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -19,13 +19,56 @@ package org.apache.spark.executor import java.net.{URLClassLoader, URL} +import org.apache.spark.util.ParentClassLoader + /** * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. + * We also make changes so user classes can come before the default classes. */ + +private[spark] trait MutableURLClassLoader extends ClassLoader { + def addURL(url: URL) + def getURLs: Array[URL] +} + +private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends MutableURLClassLoader { + + private object userClassLoader extends URLClassLoader(urls, null){ + override def addURL(url: URL) { + super.addURL(url) + } + override def findClass(name: String): Class[_] = { + super.findClass(name) + } + } + + private val parentClassLoader = new ParentClassLoader(parent) + + override def findClass(name: String): Class[_] = { + try { + userClassLoader.findClass(name) + } catch { + case e: ClassNotFoundException => { + parentClassLoader.loadClass(name) + } + } + } + + def addURL(url: URL) { + userClassLoader.addURL(url) + } + + def getURLs() = { + userClassLoader.getURLs() + } +} + private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends URLClassLoader(urls, parent) { + extends URLClassLoader(urls, parent) with MutableURLClassLoader { override def addURL(url: URL) { super.addURL(url) } } + diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala new file mode 100644 index 0000000000000..3abc12681fe9a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -0,0 +1,32 @@ +/* + * 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.util + +/** + * A class loader which makes findClass accesible to the child + */ +private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { + + override def findClass(name: String) = { + super.findClass(name) + } + + override def loadClass(name: String): Class[_] = { + super.loadClass(name) + } +} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala new file mode 100644 index 0000000000000..c40cfc0696fce --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -0,0 +1,67 @@ +/* + * 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.executor + +import java.io.File +import java.net.URLClassLoader + +import org.scalatest.FunSuite + +import org.apache.spark.TestUtils + +class ExecutorURLClassLoaderSuite extends FunSuite { + + val childClassNames = List("FakeClass1", "FakeClass2") + val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3") + val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray + val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("FakeClassDoesNotExist").newInstance() + } + } + + +} diff --git a/docs/configuration.md b/docs/configuration.md index 57bda20edcdf1..9c602402f0635 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -596,6 +596,15 @@ Apart from these, the following properties are also available, and may be useful the driver. + + spark.files.userClassPathFirst + false + + (Experimental) Whether to give user-added jars precedence over Spark's own jars when + loading classes in Executors. This feature can be used to mitigate conflicts between + Spark's dependencies and user dependencies. It is currently an experimental feature. + + spark.authenticate false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 08667aac2cd2d..694f90a83ab67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -195,6 +195,7 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( + // HTTPS is unavailable for Maven Central "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index bf73800388ebf..a30dcfdcecf27 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -26,21 +26,23 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkEnv import org.apache.spark.util.Utils - +import org.apache.spark.util.ParentClassLoader import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ - /** * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, - * used to load classes defined by the interpreter when the REPL is used - */ -class ExecutorClassLoader(classUri: String, parent: ClassLoader) -extends ClassLoader(parent) { + * used to load classes defined by the interpreter when the REPL is used. + * Allows the user to specify if user class path should be first + */ +class ExecutorClassLoader(classUri: String, parent: ClassLoader, + userClassPathFirst: Boolean) extends ClassLoader { val uri = new URI(classUri) val directory = uri.getPath + val parentLoader = new ParentClassLoader(parent) + // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { if (uri.getScheme() == "http") { @@ -49,8 +51,27 @@ extends ClassLoader(parent) { FileSystem.get(uri, new Configuration()) } } - + override def findClass(name: String): Class[_] = { + userClassPathFirst match { + case true => findClassLocally(name).getOrElse(parentLoader.loadClass(name)) + case false => { + try { + parentLoader.loadClass(name) + } catch { + case e: ClassNotFoundException => { + val classOption = findClassLocally(name) + classOption match { + case None => throw new ClassNotFoundException(name, e) + case Some(a) => a + } + } + } + } + } + } + + def findClassLocally(name: String): Option[Class[_]] = { try { val pathInDirectory = name.replace('.', '/') + ".class" val inputStream = { @@ -68,9 +89,9 @@ extends ClassLoader(parent) { } val bytes = readAndTransformClass(name, inputStream) inputStream.close() - return defineClass(name, bytes, 0, bytes.length) + Some(defineClass(name, bytes, 0, bytes.length)) } catch { - case e: Exception => throw new ClassNotFoundException(name, e) + case e: Exception => None } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala new file mode 100644 index 0000000000000..336df988a1b7f --- /dev/null +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.repl + +import java.io.File +import java.net.URLClassLoader + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import com.google.common.io.Files + +import org.apache.spark.TestUtils + +class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { + + val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") + val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") + val tempDir1 = Files.createTempDir() + val tempDir2 = Files.createTempDir() + val url1 = "file://" + tempDir1 + val urls2 = List(tempDir2.toURI.toURL).toArray + + override def beforeAll() { + childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) + parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2")) + } + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, false) + val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + } + } + +} From 9689b663a2a4947ad60795321c770052f3c637f1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 23:01:15 -0700 Subject: [PATCH 10/12] [SPARK-1390] Refactoring of matrices backed by RDDs This is to refactor interfaces for matrices backed by RDDs. It would be better if we have a clear separation of local matrices and those backed by RDDs. Right now, we have 1. `org.apache.spark.mllib.linalg.SparseMatrix`, which is a wrapper over an RDD of matrix entries, i.e., coordinate list format. 2. `org.apache.spark.mllib.linalg.TallSkinnyDenseMatrix`, which is a wrapper over RDD[Array[Double]], i.e. row-oriented format. We will see naming collision when we introduce local `SparseMatrix`, and the name `TallSkinnyDenseMatrix` is not exact if we switch to `RDD[Vector]` from `RDD[Array[Double]]`. It would be better to have "RDD" in the class name to suggest that operations may trigger jobs. The proposed names are (all under `org.apache.spark.mllib.linalg.rdd`): 1. `RDDMatrix`: trait for matrices backed by one or more RDDs 2. `CoordinateRDDMatrix`: wrapper of `RDD[(Long, Long, Double)]` 3. `RowRDDMatrix`: wrapper of `RDD[Vector]` whose rows do not have special ordering 4. `IndexedRowRDDMatrix`: wrapper of `RDD[(Long, Vector)]` whose rows are associated with indices The current code also introduces local matrices. Author: Xiangrui Meng Closes #296 from mengxr/mat and squashes the following commits: 24d8294 [Xiangrui Meng] fix for groupBy returning Iterable bfc2b26 [Xiangrui Meng] merge master 8e4f1f5 [Xiangrui Meng] Merge branch 'master' into mat 0135193 [Xiangrui Meng] address Reza's comments 03cd7e1 [Xiangrui Meng] add pca/gram to IndexedRowMatrix add toBreeze to DistributedMatrix for test simplify tests b177ff1 [Xiangrui Meng] address Matei's comments be119fe [Xiangrui Meng] rename m/n to numRows/numCols for local matrix add tests for matrices b881506 [Xiangrui Meng] rename SparkPCA/SVD to TallSkinnyPCA/SVD e7d0d4a [Xiangrui Meng] move IndexedRDDMatrixRow to IndexedRowRDDMatrix 0d1491c [Xiangrui Meng] fix test errors a85262a [Xiangrui Meng] rename RDDMatrixRow to IndexedRDDMatrixRow b8b6ac3 [Xiangrui Meng] Remove old code 4cf679c [Xiangrui Meng] port pca to RowRDDMatrix, and add multiply and covariance 7836e2f [Xiangrui Meng] initial refactoring of matrices backed by RDDs --- .../spark/examples/mllib/SparkPCA.scala | 51 --- .../spark/examples/mllib/SparkSVD.scala | 59 --- .../spark/examples/mllib/TallSkinnyPCA.scala | 64 +++ .../spark/examples/mllib/TallSkinnySVD.scala | 64 +++ .../apache/spark/mllib/linalg/Matrices.scala | 101 +++++ .../apache/spark/mllib/linalg/MatrixSVD.scala | 29 -- .../org/apache/spark/mllib/linalg/PCA.scala | 120 ------ .../org/apache/spark/mllib/linalg/SVD.scala | 395 ------------------ ...scala => SingularValueDecomposition.scala} | 9 +- .../mllib/linalg/TallSkinnyMatrixSVD.scala | 31 -- .../linalg/distributed/CoordinateMatrix.scala | 112 +++++ .../DistributedMatrix.scala} | 23 +- .../linalg/distributed/IndexedRowMatrix.scala | 148 +++++++ .../mllib/linalg/distributed/RowMatrix.scala | 344 +++++++++++++++ .../org/apache/spark/mllib/util/LAUtils.scala | 67 --- .../linalg/BreezeMatrixConversionSuite.scala} | 29 +- .../spark/mllib/linalg/MatricesSuite.scala} | 27 +- .../apache/spark/mllib/linalg/PCASuite.scala | 124 ------ .../apache/spark/mllib/linalg/SVDSuite.scala | 194 --------- .../distributed/CoordinateMatrixSuite.scala | 98 +++++ .../distributed/IndexedRowMatrixSuite.scala | 120 ++++++ .../linalg/distributed/RowMatrixSuite.scala | 173 ++++++++ 22 files changed, 1280 insertions(+), 1102 deletions(-) delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{MatrixRow.scala => SingularValueDecomposition.scala} (81%) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{SparseMatrix.scala => distributed/DistributedMatrix.scala} (60%) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala rename mllib/src/{main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala => test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala} (51%) rename mllib/src/{main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala => test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala} (58%) delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala deleted file mode 100644 index d4e08c5e12d81..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.PCA -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix -import org.apache.spark.mllib.util._ - - -/** - * Compute PCA of an example matrix. - */ -object SparkPCA { - def main(args: Array[String]) { - if (args.length != 3) { - System.err.println("Usage: SparkPCA m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "PCA", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - val m = args(2).toInt - val n = args(3).toInt - - // Make example matrix - val data = Array.tabulate(m, n) { (a, b) => - (a + 2).toDouble * (b + 1) / (1 + a + b) } - - // recover top principal component - val coeffs = new PCA().setK(1).compute(sc.makeRDD(data)) - - println("top principal component = " + coeffs.mkString(", ")) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala deleted file mode 100644 index 2933cec497b37..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix - -/** - * Compute SVD of an example matrix - * Input file should be comma separated, 1 indexed of the form - * i,j,value - * Where i is the column, j the row, and value is the matrix entry - * - * For example input file, see: - * mllib/data/als/test.data (example is 4 x 4) - */ -object SparkSVD { - def main(args: Array[String]) { - if (args.length != 4) { - System.err.println("Usage: SparkSVD m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVD", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - // Load and parse the data file - val data = sc.textFile(args(1)).map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt - 1, parts(1).toInt - 1, parts(2).toDouble) - } - val m = args(2).toInt - val n = args(3).toInt - - // recover largest singular vector - val decomposed = new SVD().setK(1).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("singular values = " + s.collect().mkString) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala new file mode 100644 index 0000000000000..a177435e606ab --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -0,0 +1,64 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the principal components of a tall-and-skinny matrix, whose rows are observations. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + */ +object TallSkinnyPCA { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnyPCA ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnyPCA") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowMatrix(rows) + + // Compute principal components. + val pc = mat.computePrincipalComponents(mat.numCols().toInt) + + println("Principal components are:\n" + pc) + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala new file mode 100644 index 0000000000000..49d09692c8e4a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -0,0 +1,64 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the singular value decomposition (SVD) of a tall-and-skinny matrix. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + */ +object TallSkinnySVD { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnySVD ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnySVD") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowMatrix(rows) + + // Compute SVD. + val svd = mat.computeSVD(mat.numCols().toInt) + + println("Singular values are " + svd.s) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala new file mode 100644 index 0000000000000..b11ba5d30fbd3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -0,0 +1,101 @@ +/* + * 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.mllib.linalg + +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} + +/** + * Trait for a local matrix. + */ +trait Matrix extends Serializable { + + /** Number of rows. */ + def numRows: Int + + /** Number of columns. */ + def numCols: Int + + /** Converts to a dense array in column major. */ + def toArray: Array[Double] + + /** Converts to a breeze matrix. */ + private[mllib] def toBreeze: BM[Double] + + /** Gets the (i, j)-th element. */ + private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + + override def toString: String = toBreeze.toString() +} + +/** + * Column-majored dense matrix. + * The entry values are stored in a single array of doubles with columns listed in sequence. + * For example, the following matrix + * {{{ + * 1.0 2.0 + * 3.0 4.0 + * 5.0 6.0 + * }}} + * is stored as `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ +class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { + + require(values.length == numRows * numCols) + + override def toArray: Array[Double] = values + + private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.Matrix]]. + */ +object Matrices { + + /** + * Creates a column-majored dense matrix. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ + def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { + new DenseMatrix(numRows, numCols, values) + } + + /** + * Creates a Matrix instance from a breeze matrix. + * @param breeze a breeze matrix + * @return a Matrix instance + */ + private[mllib] def fromBreeze(breeze: BM[Double]): Matrix = { + breeze match { + case dm: BDM[Double] => + require(dm.majorStride == dm.rows, + "Do not support stride size different from the number of rows.") + new DenseMatrix(dm.rows, dm.cols, dm.data) + case _ => + throw new UnsupportedOperationException( + s"Do not support conversion from type ${breeze.getClass.getName}.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala deleted file mode 100644 index 319f82b449096..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -/** - * Class that represents the SV decomposition of a matrix - * - * @param U such that A = USV^T - * @param S such that A = USV^T - * @param V such that A = USV^T - */ -case class MatrixSVD(val U: SparseMatrix, - val S: SparseMatrix, - val V: SparseMatrix) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala deleted file mode 100644 index fe5b3f6c7e463..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import org.apache.spark.rdd.RDD - - -import org.jblas.DoubleMatrix - - -/** - * Class used to obtain principal components - */ -class PCA { - private var k = 1 - - /** - * Set the number of top-k principle components to return - */ - def setK(k: Int): PCA = { - this.k = k - this - } - - /** - * Compute PCA using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Compute PCA using the parameters currently set - * See computePCA() for more details - */ - def compute(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform PCA on - * @return An nxk matrix with principal components in columns. Columns are inner arrays - */ - private def computePCA(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - val m = matrix.m - val n = matrix.n - - if (m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a well-formed matrix: m=$m n=$n") - } - - computePCA(matrix.rows.map(_.data)) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform pca on - * @return An nxk matrix of principal components - */ - private def computePCA(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - val n = matrix.first.size - - // compute column sums and normalize matrix - val colSumsTemp = matrix.map((_, 1)).fold((Array.ofDim[Double](n), 0)) { - (a, b) => - val am = new DoubleMatrix(a._1) - val bm = new DoubleMatrix(b._1) - am.addi(bm) - (a._1, a._2 + b._2) - } - - val m = colSumsTemp._2 - val colSums = colSumsTemp._1.map(x => x / m) - - val data = matrix.map { - x => - val row = Array.ofDim[Double](n) - var i = 0 - while (i < n) { - row(i) = x(i) - colSums(i) - i += 1 - } - row - } - - val (u, s, v) = new SVD().setK(k).compute(data) - v - } -} - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala deleted file mode 100644 index 0d97b7d92f155..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ /dev/null @@ -1,395 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -/** - * Class used to obtain singular value decompositions - */ -class SVD { - private var k = 1 - private var computeU = true - - // All singular values smaller than rCond * sigma(0) - // are treated as zero, where sigma(0) is the largest singular value. - private var rCond = 1e-9 - - /** - * Set the number of top-k singular vectors to return - */ - def setK(k: Int): SVD = { - this.k = k - this - } - - /** - * Sets the reciprocal condition number (rCond). All singular values - * smaller than rCond * sigma(0) are treated as zero, - * where sigma(0) is the largest singular value. - */ - def setReciprocalConditionNumber(smallS: Double): SVD = { - this.rCond = smallS - this - } - - /** - * Should U be computed? - */ - def setComputeU(compU: Boolean): SVD = { - this.computeU = compU - this - } - - /** - * Compute SVD using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - denseSVD(matrix) - } - - /** - * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T - * U is a row-by-row dense matrix - * S is a simple double array of singular values - * V is a 2d array matrix - * See [[denseSVD]] for more documentation - */ - def compute(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - denseSVD(matrix) - } - - /** - * See full paramter definition of sparseSVD for more description. - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - def compute(matrix: SparseMatrix): MatrixSVD = { - sparseSVD(matrix) - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * @param matrix dense matrix to factorize - * @return See [[TallSkinnyMatrixSVD]] for the output matrices and arrays - */ - private def denseSVD(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Request up to n singular values n=$n k=$k") - } - - val rowIndices = matrix.rows.map(_.i) - - // compute SVD - val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) - - if (computeU) { - // prep u for returning - val retU = TallSkinnyDenseMatrix( - u.zip(rowIndices).map { - case (row, i) => MatrixRow(i, row) - }, - m, - k) - - TallSkinnyMatrixSVD(retU, sigma, v) - } else { - TallSkinnyMatrixSVD(null, sigma, v) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * The return values are as lean as possible: an RDD of rows for U, - * a simple array for sigma, and a dense 2d matrix array for V - * - * @param matrix dense matrix to factorize - * @return Three matrices: U, S, V such that A = USV^T - */ - private def denseSVD(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - val n = matrix.first.size - - if (k < 1 || k > n) { - throw new IllegalArgumentException( - "Request up to n singular values k=$k n=$n") - } - - // Compute A^T A - val fullata = matrix.mapPartitions { - iter => - val localATA = Array.ofDim[Double](n, n) - while (iter.hasNext) { - val row = iter.next() - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - localATA(i)(j) += row(i) * row(j) - j += 1 - } - i += 1 - } - } - Iterator(localATA) - }.fold(Array.ofDim[Double](n, n)) { - (a, b) => - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - a(i)(j) += b(i)(j) - j += 1 - } - i += 1 - } - a - } - - // Construct jblas A^T A locally - val ata = new DoubleMatrix(fullata) - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x / svd(1).get(0) > rCond) - - val sk = Math.min(k, sigmas.size) - val sigma = sigmas.take(sk) - - // prepare V for returning - val retV = Array.tabulate(n, sk)((i, j) => V.get(i, j)) - - if (computeU) { - // Compute U as U = A V S^-1 - // Compute VS^-1 - val vsinv = new DoubleMatrix(Array.tabulate(n, sk)((i, j) => V.get(i, j) / sigma(j))) - val retU = matrix.map { - x => - val v = new DoubleMatrix(Array(x)) - v.mmul(vsinv).data - } - (retU, sigma, retV) - } else { - (null, sigma, retV) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny sparse matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require O(n^2) doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * All input and output is expected in sparse matrix format, 0-indexed - * as tuples of the form ((i,j),value) all in RDDs using the - * SparseMatrix class - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - private def sparseSVD(matrix: SparseMatrix): MatrixSVD = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Must request up to n singular values") - } - - // Compute A^T A, assuming rows are sparse enough to fit in memory - val rows = data.map(entry => - (entry.i, (entry.j, entry.mval))).groupByKey() - val emits = rows.flatMap { - case (rowind, cols) => - cols.flatMap { - case (colind1, mval1) => - cols.map { - case (colind2, mval2) => - ((colind1, colind2), mval1 * mval2) - } - } - }.reduceByKey(_ + _) - - // Construct jblas A^T A locally - val ata = DoubleMatrix.zeros(n, n) - for (entry <- emits.collect()) { - ata.put(entry._1._1, entry._1._2, entry._2) - } - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - // This will be updated to rcond - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) - - if (sigmas.size < k) { - throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) - } - - val sigma = sigmas.take(k) - - val sc = data.sparkContext - - // prepare V for returning - val retVdata = sc.makeRDD( - Array.tabulate(V.rows, sigma.length) { - (i, j) => - MatrixEntry(i, j, V.get(i, j)) - }.flatten) - val retV = SparseMatrix(retVdata, V.rows, sigma.length) - - val retSdata = sc.makeRDD(Array.tabulate(sigma.length) { - x => MatrixEntry(x, x, sigma(x)) - }) - - val retS = SparseMatrix(retSdata, sigma.length, sigma.length) - - // Compute U as U = A V S^-1 - // turn V S^-1 into an RDD as a sparse matrix - val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length) { - (i, j) => ((i, j), V.get(i, j) / sigma(j)) - }.flatten) - - if (computeU) { - // Multiply A by VS^-1 - val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) - val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - val retUdata = aCols.join(bRows).map { - case (key, ((rowInd, rowVal), (colInd, colVal))) => - ((rowInd, colInd), rowVal * colVal) - }.reduceByKey(_ + _).map { - case ((row, col), mval) => MatrixEntry(row, col, mval) - } - - val retU = SparseMatrix(retUdata, m, sigma.length) - MatrixSVD(retU, retS, retV) - } else { - MatrixSVD(null, retS, retV) - } - } -} - -/** - * Top-level methods for calling sparse Singular Value Decomposition - * NOTE: All matrices are 0-indexed - */ -object SVD { - def main(args: Array[String]) { - if (args.length < 8) { - println("Usage: SVD " + - " ") - System.exit(1) - } - - val (master, inputFile, m, n, k, output_u, output_s, output_v) = - (args(0), args(1), args(2).toInt, args(3).toInt, - args(4).toInt, args(5), args(6), args(7)) - - val sc = new SparkContext(master, "SVD") - - val rawData = sc.textFile(inputFile) - val data = rawData.map { - line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) - } - - val decomposed = new SVD().setK(k).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("Computed " + s.collect().length + " singular values and vectors") - u.saveAsTextFile(output_u) - s.saveAsTextFile(output_s) - v.saveAsTextFile(output_v) - System.exit(0) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala similarity index 81% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 2608a67bfe260..46b105457430c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,10 +17,5 @@ package org.apache.spark.mllib.linalg -/** - * Class that represents a row of a dense matrix - * - * @param i row index (0 indexing used) - * @param data entries of the row - */ -case class MatrixRow(val i: Int, val data: Array[Double]) +/** Represents singular value decomposition (SVD) factors. */ +case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala deleted file mode 100644 index b3a450e92394e..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -/** - * Class that represents the singular value decomposition of a matrix - * - * @param U such that A = USV^T is a TallSkinnyDenseMatrix - * @param S such that A = USV^T is a simple double array - * @param V such that A = USV^T, V is a 2d array matrix that holds - * singular vectors in columns. Columns are inner arrays - * i.e. V(i)(j) is standard math notation V_{ij} - */ -case class TallSkinnyMatrixSVD(val U: TallSkinnyDenseMatrix, - val S: Array[Double], - val V: Array[Array[Double]]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala new file mode 100644 index 0000000000000..9194f657494b2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -0,0 +1,112 @@ +/* + * 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.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors + +/** + * Represents an entry in an distributed matrix. + * @param i row index + * @param j column index + * @param value value of the entry + */ +case class MatrixEntry(i: Long, j: Long, value: Double) + +/** + * Represents a matrix in coordinate format. + * + * @param entries matrix entries + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the max column index plus one. + */ +class CoordinateMatrix( + val entries: RDD[MatrixEntry], + private var nRows: Long, + private var nCols: Long) extends DistributedMatrix { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(entries: RDD[MatrixEntry]) = this(entries, 0L, 0L) + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (nCols <= 0L) { + computeSize() + } + nCols + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (nRows <= 0L) { + computeSize() + } + nRows + } + + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ + def toIndexedRowMatrix(): IndexedRowMatrix = { + val nl = numCols() + if (nl > Int.MaxValue) { + sys.error(s"Cannot convert to a row-oriented format because the number of columns $nl is " + + "too large.") + } + val n = nl.toInt + val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) + .groupByKey() + .map { case (i, vectorEntries) => + IndexedRow(i, Vectors.sparse(n, vectorEntries.toSeq)) + } + new IndexedRowMatrix(indexedRows, numRows(), n) + } + + /** + * Converts to RowMatrix, dropping row indices after grouping by row index. + * The number of columns must be within the integer range. + */ + def toRowMatrix(): RowMatrix = { + toIndexedRowMatrix().toRowMatrix() + } + + /** Determines the size by computing the max row/column index. */ + private def computeSize() { + // Reduce will throw an exception if `entries` is empty. + val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => + (math.max(i1, i2), math.max(j1, j2)) + } + // There may be empty columns at the very right and empty rows at the very bottom. + nRows = math.max(nRows, m1 + 1L) + nCols = math.max(nCols, n1 + 1L) + } + + /** Collects data and assembles a local matrix. */ + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + entries.collect().foreach { case MatrixEntry(i, j, value) => + mat(i.toInt, j.toInt) = value + } + mat + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala similarity index 60% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index cbd1a2a5a4bd8..13f72a3c724ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -15,16 +15,23 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg +package org.apache.spark.mllib.linalg.distributed -import org.apache.spark.rdd.RDD +import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.mllib.linalg.Matrix /** - * Class that represents a sparse matrix - * - * @param data RDD of nonzero entries - * @param m number of rows - * @param n numner of columns + * Represents a distributively stored matrix backed by one or more RDDs. */ -case class SparseMatrix(val data: RDD[MatrixEntry], val m: Int, val n: Int) +trait DistributedMatrix extends Serializable { + + /** Gets or computes the number of rows. */ + def numRows(): Long + + /** Gets or computes the number of columns. */ + def numCols(): Long + + /** Collects data and assembles a local dense breeze matrix (for test only). */ + private[mllib] def toBreeze(): BDM[Double] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala new file mode 100644 index 0000000000000..e110f070bd7c1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -0,0 +1,148 @@ +/* + * 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.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.SingularValueDecomposition + +/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +case class IndexedRow(index: Long, vector: Vector) + +/** + * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with + * indexed rows. + * + * @param rows indexed rows of this matrix + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. + */ +class IndexedRowMatrix( + val rows: RDD[IndexedRow], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[IndexedRow]) = this(rows, 0L, 0) + + override def numCols(): Long = { + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().vector.size + } + nCols + } + + override def numRows(): Long = { + if (nRows <= 0L) { + // Reduce will throw an exception if `rows` is empty. + nRows = rows.map(_.index).reduce(math.max) + 1L + } + nRows + } + + /** + * Drops row indices and converts this matrix to a + * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]]. + */ + def toRowMatrix(): RowMatrix = { + new RowMatrix(rows.map(_.vector), 0L, nCols) + } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is an [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]] of size m x k that + * satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a local Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[IndexedRowMatrix, Matrix] = { + val indices = rows.map(_.index) + val svd = toRowMatrix().computeSVD(k, computeU, rCond) + val U = if (computeU) { + val indexedRows = indices.zip(svd.U.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } else { + null + } + SingularValueDecomposition(U, svd.s, svd.V) + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return an IndexedRowMatrix representing the product, which preserves partitioning + */ + def multiply(B: Matrix): IndexedRowMatrix = { + val mat = toRowMatrix().multiply(B) + val indexedRows = rows.map(_.index).zip(mat.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + toRowMatrix().computeGramianMatrix() + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + rows.collect().foreach { case IndexedRow(rowIndex, vector) => + val i = rowIndex.toInt + vector.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + } + mat + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala new file mode 100644 index 0000000000000..f59811f18a68f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -0,0 +1,344 @@ +/* + * 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.mllib.linalg.distributed + +import java.util + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.numerics.{sqrt => brzSqrt} +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.mllib.linalg._ +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging + +/** + * Represents a row-oriented distributed Matrix with no meaningful row indices. + * + * @param rows rows stored as an RDD[Vector] + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the number of records in the RDD `rows`. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. + */ +class RowMatrix( + val rows: RDD[Vector], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix with Logging { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[Vector]) = this(rows, 0L, 0) + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().size + } + nCols + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (nRows <= 0L) { + nRows = rows.count() + if (nRows == 0L) { + sys.error("Cannot determine the number of rows because it is not specified in the " + + "constructor and the rows RDD is empty.") + } + } + nRows + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + val n = numCols().toInt + val nt: Int = n * (n + 1) / 2 + + // Compute the upper triangular part of the gram matrix. + val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( + seqOp = (U, v) => { + RowMatrix.dspr(1.0, v, U.data) + U + }, + combOp = (U1, U2) => U1 += U2 + ) + + RowMatrix.triuToFull(n, GU.data) + } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is a RowMatrix of size m x k that satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[RowMatrix, Matrix] = { + val n = numCols().toInt + require(k > 0 && k <= n, s"Request up to n singular values k=$k n=$n.") + + val G = computeGramianMatrix() + + // TODO: Use sparse SVD instead. + val (u: BDM[Double], sigmaSquares: BDV[Double], v: BDM[Double]) = + brzSvd(G.toBreeze.asInstanceOf[BDM[Double]]) + val sigmas: BDV[Double] = brzSqrt(sigmaSquares) + + // Determine effective rank. + val sigma0 = sigmas(0) + val threshold = rCond * sigma0 + var i = 0 + while (i < k && sigmas(i) >= threshold) { + i += 1 + } + val sk = i + + if (sk < k) { + logWarning(s"Requested $k singular values but only found $sk nonzeros.") + } + + val s = Vectors.dense(util.Arrays.copyOfRange(sigmas.data, 0, sk)) + val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + + if (computeU) { + // N = Vk * Sk^{-1} + val N = new BDM[Double](n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + var i = 0 + var j = 0 + while (j < sk) { + i = 0 + val sigma = sigmas(j) + while (i < n) { + N(i, j) /= sigma + i += 1 + } + j += 1 + } + val U = this.multiply(Matrices.fromBreeze(N)) + SingularValueDecomposition(U, s, V) + } else { + SingularValueDecomposition(null, s, V) + } + } + + /** + * Computes the covariance matrix, treating each row as an observation. + * @return a local dense matrix of size n x n + */ + def computeCovariance(): Matrix = { + val n = numCols().toInt + + if (n > 10000) { + val mem = n * n * java.lang.Double.SIZE / java.lang.Byte.SIZE + logWarning(s"The number of columns $n is greater than 10000! " + + s"We need at least $mem bytes of memory.") + } + + val (m, mean) = rows.aggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( + seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), + combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) + ) + + // Update _m if it is not set, or verify its value. + if (nRows <= 0L) { + nRows = m + } else { + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") + } + + mean :/= m.toDouble + + // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is + // large but Cov(X, Y) is small, but it is good for sparse computation. + // TODO: find a fast and stable way for sparse data. + + val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + + var i = 0 + var j = 0 + val m1 = m - 1.0 + var alpha = 0.0 + while (i < n) { + alpha = m / m1 * mean(i) + j = 0 + while (j < n) { + G(i, j) = G(i, j) / m1 - alpha * mean(j) + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(G) + } + + /** + * Computes the top k principal components. + * Rows correspond to observations and columns correspond to variables. + * The principal components are stored a local matrix of size n-by-k. + * Each column corresponds for one principal component, + * and the columns are in descending order of component variance. + * + * @param k number of top principal components. + * @return a matrix of size n-by-k, whose columns are principal components + */ + def computePrincipalComponents(k: Int): Matrix = { + val n = numCols().toInt + require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") + + val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] + + val (u: BDM[Double], _, _) = brzSvd(Cov) + + if (k == n) { + Matrices.dense(n, k, u.data) + } else { + Matrices.dense(n, k, util.Arrays.copyOfRange(u.data, 0, n * k)) + } + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return a [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] representing the product, + * which preserves partitioning + */ + def multiply(B: Matrix): RowMatrix = { + val n = numCols().toInt + require(n == B.numRows, s"Dimension mismatch: $n vs ${B.numRows}") + + require(B.isInstanceOf[DenseMatrix], + s"Only support dense matrix at this time but found ${B.getClass.getName}.") + + val Bb = rows.context.broadcast(B) + val AB = rows.mapPartitions({ iter => + val Bi = Bb.value.toBreeze.asInstanceOf[BDM[Double]] + iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) + }, preservesPartitioning = true) + + new RowMatrix(AB, nRows, B.numCols) + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + var i = 0 + rows.collect().foreach { v => + v.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + i += 1 + } + mat + } +} + +object RowMatrix { + + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. + * + * @param U the upper triangular part of the matrix packed in an array (column major) + */ + private def dspr(alpha: Double, v: Vector, U: Array[Double]): Unit = { + // TODO: Find a better home (breeze?) for this method. + val n = v.size + v match { + case dv: DenseVector => + blas.dspr("U", n, 1.0, dv.values, 1, U) + case sv: SparseVector => + val indices = sv.indices + val values = sv.values + val nnz = indices.length + var colStartIdx = 0 + var prevCol = 0 + var col = 0 + var j = 0 + var i = 0 + var av = 0.0 + while (j < nnz) { + col = indices(j) + // Skip empty columns. + colStartIdx += (col - prevCol) * (col + prevCol + 1) / 2 + col = indices(j) + av = alpha * values(j) + i = 0 + while (i <= j) { + U(colStartIdx + indices(i)) += av * values(i) + i += 1 + } + j += 1 + prevCol = col + } + } + } + + /** + * Fills a full square matrix from its upper triangular part. + */ + private def triuToFull(n: Int, U: Array[Double]): Matrix = { + val G = new BDM[Double](n, n) + + var row = 0 + var col = 0 + var idx = 0 + var value = 0.0 + while (col < n) { + row = 0 + while (row < col) { + value = U(idx) + G(row, col) = value + G(col, row) = value + idx += 1 + row += 1 + } + G(col, col) = U(idx) + idx += 1 + col +=1 + } + + Matrices.dense(n, n, G.data) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala deleted file mode 100644 index 87aac347579c7..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.util - -import org.apache.spark.SparkContext._ - -import org.apache.spark.mllib.linalg._ - -/** - * Helper methods for linear algebra - */ -object LAUtils { - /** - * Convert a SparseMatrix into a TallSkinnyDenseMatrix - * - * @param sp Sparse matrix to be converted - * @return dense version of the input - */ - def sparseToTallSkinnyDense(sp: SparseMatrix): TallSkinnyDenseMatrix = { - val m = sp.m - val n = sp.n - val rows = sp.data.map(x => (x.i, (x.j, x.mval))).groupByKey().map { - case (i, cols) => - val rowArray = Array.ofDim[Double](n) - var j = 0 - val colsItr = cols.iterator - while (colsItr.hasNext) { - val element = colsItr.next - rowArray(element._1) = element._2 - j += 1 - } - MatrixRow(i, rowArray) - } - TallSkinnyDenseMatrix(rows, m, n) - } - - /** - * Convert a TallSkinnyDenseMatrix to a SparseMatrix - * - * @param a matrix to be converted - * @return sparse version of the input - */ - def denseToSparse(a: TallSkinnyDenseMatrix): SparseMatrix = { - val m = a.m - val n = a.n - val data = a.rows.flatMap { - mrow => Array.tabulate(n)(j => MatrixEntry(mrow.i, j, mrow.data(j))) - .filter(x => x.mval != 0) - } - SparseMatrix(data, m, n) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala similarity index 51% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index 416996fcbe760..82d49c76ed02b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -17,11 +17,24 @@ package org.apache.spark.mllib.linalg -/** - * Class that represents an entry in a sparse matrix of doubles. - * - * @param i row index (0 indexing used) - * @param j column index (0 indexing used) - * @param mval value of entry in matrix - */ -case class MatrixEntry(val i: Int, val j: Int, val mval: Double) +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +class BreezeMatrixConversionSuite extends FunSuite { + test("dense matrix to breeze") { + val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") + } + + test("dense breeze matrix to matrix") { + val breeze = new BDM[Double](3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[DenseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala similarity index 58% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index e4ef3c58e8680..9c66b4db9f16b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -17,14 +17,23 @@ package org.apache.spark.mllib.linalg -import org.apache.spark.rdd.RDD +import org.scalatest.FunSuite +class MatricesSuite extends FunSuite { + test("dense matrix construction") { + val m = 3 + val n = 2 + val values = Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0) + val mat = Matrices.dense(m, n, values).asInstanceOf[DenseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.toArray.eq(values), "toArray should not copy data") + } -/** - * Class that represents a dense matrix - * - * @param rows RDD of rows - * @param m number of rows - * @param n number of columns - */ -case class TallSkinnyDenseMatrix(val rows: RDD[MatrixRow], val m: Int, val n: Int) + test("dense matrix construction with wrong dimension") { + intercept[RuntimeException] { + Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala deleted file mode 100644 index 5e5086b1bf73e..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class PCASuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-3 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val ret = DoubleMatrix.zeros(matrix.m, matrix.n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3)), 3) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("sparse matrix full rank matrix pca") { - val m = 5 - val n = 3 - // the entry that gets dropped is zero to test sparse support - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten.drop(1) - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("truncated matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), - (1,0,-0.1448), (1,1,0.7483), - (2,0,0.9553), (2,1,-0.0649)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val k = 2 - val coeffs = new DoubleMatrix(new PCA().setK(k).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,k)), coeffs) - } -} - - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala deleted file mode 100644 index 20e2b0f84be06..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ /dev/null @@ -1,194 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class SVDSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - val ret = DoubleMatrix.zeros(m, n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(n).compute(a) - val u = decomposed.U - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("dense full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val decomposed = new SVD().setK(n).setComputeU(true).compute(a) - val u = LAUtils.denseToSparse(decomposed.U) - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(LAUtils.denseToSparse(a)) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = DoubleMatrix.diag(new DoubleMatrix(s)) - val retv = new DoubleMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("rank one matrix svd") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) => - MatrixEntry(a, b, 1.0) }.flatten ) - val k = 1 - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - assert(retrank == 1, "rank returned not one") - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("truncated with k") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1)/(1 + a + b)) }.flatten ) - val a = SparseMatrix(data, m, n) - - val k = 1 // only one svalue above this - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - assert(retrank == 1, "rank returned not one") - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala new file mode 100644 index 0000000000000..cd45438fb628f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -0,0 +1,98 @@ +/* + * 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.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors + +class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 5 + val n = 4 + var mat: CoordinateMatrix = _ + + override def beforeAll() { + super.beforeAll() + val entries = sc.parallelize(Seq( + (0, 0, 1.0), + (0, 1, 2.0), + (1, 1, 3.0), + (1, 2, 4.0), + (2, 2, 5.0), + (2, 3, 6.0), + (3, 0, 7.0), + (3, 3, 8.0), + (4, 1, 9.0)), 3).map { case (i, j, value) => + MatrixEntry(i, j, value) + } + mat = new CoordinateMatrix(entries) + } + + test("size") { + assert(mat.numRows() === m) + assert(mat.numCols() === n) + } + + test("empty entries") { + val entries = sc.parallelize(Seq[MatrixEntry](), 1) + val emptyMat = new CoordinateMatrix(entries) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toBreeze") { + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(mat.toBreeze() === expected) + } + + test("toIndexedRowMatrix") { + val indexedRowMatrix = mat.toIndexedRowMatrix() + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(indexedRowMatrix.toBreeze() === expected) + } + + test("toRowMatrix") { + val rowMatrix = mat.toRowMatrix() + val rows = rowMatrix.rows.collect().toSet + val expected = Set( + Vectors.dense(1.0, 2.0, 0.0, 0.0), + Vectors.dense(0.0, 3.0, 4.0, 0.0), + Vectors.dense(0.0, 0.0, 5.0, 6.0), + Vectors.dense(7.0, 0.0, 0.0, 8.0), + Vectors.dense(0.0, 9.0, 0.0, 0.0)) + assert(rows === expected) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala new file mode 100644 index 0000000000000..f7c46f23b746d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -0,0 +1,120 @@ +/* + * 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.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrices, Vectors} + +class IndexedRowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val data = Seq( + (0L, Vectors.dense(0.0, 1.0, 2.0)), + (1L, Vectors.dense(3.0, 4.0, 5.0)), + (3L, Vectors.dense(9.0, 0.0, 1.0)) + ).map(x => IndexedRow(x._1, x._2)) + var indexedRows: RDD[IndexedRow] = _ + + override def beforeAll() { + super.beforeAll() + indexedRows = sc.parallelize(data, 2) + } + + test("size") { + val mat1 = new IndexedRowMatrix(indexedRows) + assert(mat1.numRows() === m) + assert(mat1.numCols() === n) + + val mat2 = new IndexedRowMatrix(indexedRows, 5, 0) + assert(mat2.numRows() === 5) + assert(mat2.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[IndexedRow](), 1) + val mat = new IndexedRowMatrix(rows) + intercept[RuntimeException] { + mat.numRows() + } + intercept[RuntimeException] { + mat.numCols() + } + } + + test("toBreeze") { + val mat = new IndexedRowMatrix(indexedRows) + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (0.0, 0.0, 0.0), + (9.0, 0.0, 1.0)) + assert(mat.toBreeze() === expected) + } + + test("toRowMatrix") { + val idxRowMat = new IndexedRowMatrix(indexedRows) + val rowMat = idxRowMat.toRowMatrix() + assert(rowMat.numCols() === n) + assert(rowMat.numRows() === 3, "should drop empty rows") + assert(rowMat.rows.collect().toSeq === data.map(_.vector).toSeq) + } + + test("multiply a local matrix") { + val A = new IndexedRowMatrix(indexedRows) + val B = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val C = A.multiply(B) + val localA = A.toBreeze() + val localC = C.toBreeze() + val expected = localA * B.toBreeze.asInstanceOf[BDM[Double]] + assert(localC === expected) + } + + test("gram") { + val A = new IndexedRowMatrix(indexedRows) + val G = A.computeGramianMatrix() + val expected = BDM( + (90.0, 12.0, 24.0), + (12.0, 17.0, 22.0), + (24.0, 22.0, 30.0)) + assert(G.toBreeze === expected) + } + + test("svd") { + val A = new IndexedRowMatrix(indexedRows) + val svd = A.computeSVD(n, computeU = true) + assert(svd.U.isInstanceOf[IndexedRowMatrix]) + val localA = A.toBreeze() + val U = svd.U.toBreeze() + val s = svd.s.toBreeze.asInstanceOf[BDV[Double]] + val V = svd.V.toBreeze.asInstanceOf[BDM[Double]] + assert(closeToZero(U.t * U - BDM.eye[Double](n))) + assert(closeToZero(V.t * V - BDM.eye[Double](n))) + assert(closeToZero(U * brzDiag(s) * V.t - localA)) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } +} + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala new file mode 100644 index 0000000000000..71ee8e8a4f6fd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -0,0 +1,173 @@ +/* + * 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.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} + +class RowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val arr = Array(0.0, 3.0, 6.0, 9.0, 1.0, 4.0, 7.0, 0.0, 2.0, 5.0, 8.0, 1.0) + val denseData = Seq( + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(3.0, 4.0, 5.0), + Vectors.dense(6.0, 7.0, 8.0), + Vectors.dense(9.0, 0.0, 1.0) + ) + val sparseData = Seq( + Vectors.sparse(3, Seq((1, 1.0), (2, 2.0))), + Vectors.sparse(3, Seq((0, 3.0), (1, 4.0), (2, 5.0))), + Vectors.sparse(3, Seq((0, 6.0), (1, 7.0), (2, 8.0))), + Vectors.sparse(3, Seq((0, 9.0), (2, 1.0))) + ) + + val principalComponents = BDM( + (0.0, 1.0, 0.0), + (math.sqrt(2.0) / 2.0, 0.0, math.sqrt(2.0) / 2.0), + (math.sqrt(2.0) / 2.0, 0.0, - math.sqrt(2.0) / 2.0)) + + var denseMat: RowMatrix = _ + var sparseMat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + denseMat = new RowMatrix(sc.parallelize(denseData, 2)) + sparseMat = new RowMatrix(sc.parallelize(sparseData, 2)) + } + + test("size") { + assert(denseMat.numRows() === m) + assert(denseMat.numCols() === n) + assert(sparseMat.numRows() === m) + assert(sparseMat.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[Vector](), 1) + val emptyMat = new RowMatrix(rows) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toBreeze") { + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (6.0, 7.0, 8.0), + (9.0, 0.0, 1.0)) + for (mat <- Seq(denseMat, sparseMat)) { + assert(mat.toBreeze() === expected) + } + } + + test("gram") { + val expected = + Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.computeGramianMatrix() + assert(G.toBreeze === expected.toBreeze) + } + } + + test("svd of a full-rank matrix") { + for (mat <- Seq(denseMat, sparseMat)) { + val localMat = mat.toBreeze() + val (localU, localSigma, localVt) = brzSvd(localMat) + val localV: BDM[Double] = localVt.t.toDenseMatrix + for (k <- 1 to n) { + val svd = mat.computeSVD(k, computeU = true) + val U = svd.U + val s = svd.s + val V = svd.V + assert(U.numRows() === m) + assert(U.numCols() === k) + assert(s.size === k) + assert(V.numRows === n) + assert(V.numCols === k) + assertColumnEqualUpToSign(U.toBreeze(), localU, k) + assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + } + val svdWithoutU = mat.computeSVD(n) + assert(svdWithoutU.U === null) + } + } + + test("svd of a low-rank matrix") { + val rows = sc.parallelize(Array.fill(4)(Vectors.dense(1.0, 1.0)), 2) + val mat = new RowMatrix(rows, 4, 2) + val svd = mat.computeSVD(2, computeU = true) + assert(svd.s.size === 1, "should not return zero singular values") + assert(svd.U.numRows() === 4) + assert(svd.U.numCols() === 1) + assert(svd.V.numRows === 2) + assert(svd.V.numCols === 1) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } + + def closeToZero(v: BDV[Double]): Boolean = { + brzNorm(v, 1.0) < 1e-6 + } + + def assertColumnEqualUpToSign(A: BDM[Double], B: BDM[Double], k: Int) { + assert(A.rows === B.rows) + for (j <- 0 until k) { + val aj = A(::, j) + val bj = B(::, j) + assert(closeToZero(aj - bj) || closeToZero(aj + bj), + s"The $j-th columns mismatch: $aj and $bj") + } + } + + test("pca") { + for (mat <- Seq(denseMat, sparseMat); k <- 1 to n) { + val pc = denseMat.computePrincipalComponents(k) + assert(pc.numRows === n) + assert(pc.numCols === k) + assertColumnEqualUpToSign(pc.toBreeze.asInstanceOf[BDM[Double]], principalComponents, k) + } + } + + test("multiply a local matrix") { + val B = Matrices.dense(n, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val AB = mat.multiply(B) + assert(AB.numRows() === m) + assert(AB.numCols() === 2) + assert(AB.rows.collect().toSeq === Seq( + Vectors.dense(5.0, 14.0), + Vectors.dense(14.0, 50.0), + Vectors.dense(23.0, 86.0), + Vectors.dense(2.0, 32.0) + )) + } + } +} From 87bd1f9ef7d547ee54a8a83214b45462e0751efb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 9 Apr 2014 01:14:46 -0700 Subject: [PATCH 11/12] SPARK-1093: Annotate developer and experimental API's This patch marks some existing classes as private[spark] and adds two types of API annotations: - `EXPERIMENTAL API` = experimental user-facing module - `DEVELOPER API - UNSTABLE` = developer-facing API that might change There is some discussion of the different mechanisms for doing this here: https://issues.apache.org/jira/browse/SPARK-1081 I was pretty aggressive with marking things private. Keep in mind that if we want to open something up in the future we can, but we can never reduce visibility. A few notes here: - In the past we've been inconsistent with the visiblity of the X-RDD classes. This patch marks them private whenever there is an existing function in RDD that can directly creat them (e.g. CoalescedRDD and rdd.coalesce()). One trade-off here is users can't subclass them. - Noted that compression and serialization formats don't have to be wire compatible across versions. - Compression codecs and serialization formats are semi-private as users typically don't instantiate them directly. - Metrics sources are made private - user only interacts with them through Spark's reflection Author: Patrick Wendell Author: Andrew Or Closes #274 from pwendell/private-apis and squashes the following commits: 44179e4 [Patrick Wendell] Merge remote-tracking branch 'apache-github/master' into private-apis 042c803 [Patrick Wendell] spark.annotations -> spark.annotation bfe7b52 [Patrick Wendell] Adding experimental for approximate counts 8d0c873 [Patrick Wendell] Warning in SparkEnv 99b223a [Patrick Wendell] Cleaning up annotations e849f64 [Patrick Wendell] Merge pull request #2 from andrewor14/annotations 982a473 [Andrew Or] Generalize jQuery matching for non Spark-core API docs a01c076 [Patrick Wendell] Merge pull request #1 from andrewor14/annotations c1bcb41 [Andrew Or] DeveloperAPI -> DeveloperApi 0d48908 [Andrew Or] Comments and new lines (minor) f3954e0 [Andrew Or] Add identifier tags in comments to work around scaladocs bug 99192ef [Andrew Or] Dynamically add badges based on annotations 824011b [Andrew Or] Add support for injecting arbitrary JavaScript to API docs 037755c [Patrick Wendell] Some changes after working with andrew or f7d124f [Patrick Wendell] Small fixes c318b24 [Patrick Wendell] Use CSS styles e4c76b9 [Patrick Wendell] Logging f390b13 [Patrick Wendell] Better visibility for workaround constructors d6b0afd [Patrick Wendell] Small chang to existing constructor 403ba52 [Patrick Wendell] Style fix 870a7ba [Patrick Wendell] Work around for SI-8479 7fb13b2 [Patrick Wendell] Changes to UnionRDD and EmptyRDD 4a9e90c [Patrick Wendell] EXPERIMENTAL API --> EXPERIMENTAL c581dce [Patrick Wendell] Changes after building against Shark. 8452309 [Patrick Wendell] Style fixes 1ed27d2 [Patrick Wendell] Formatting and coloring of badges cd7a465 [Patrick Wendell] Code review feedback 2f706f1 [Patrick Wendell] Don't use floats 542a736 [Patrick Wendell] Small fixes cf23ec6 [Patrick Wendell] Marking GraphX as alpha d86818e [Patrick Wendell] Another naming change 5a76ed6 [Patrick Wendell] More visiblity clean-up 42c1f09 [Patrick Wendell] Using better labels 9d48cbf [Patrick Wendell] Initial pass --- .../scala/org/apache/spark/Aggregator.scala | 3 + .../scala/org/apache/spark/Dependency.scala | 11 +++ .../scala/org/apache/spark/FutureAction.scala | 7 ++ .../apache/spark/InterruptibleIterator.scala | 2 +- .../main/scala/org/apache/spark/Logging.scala | 7 ++ .../apache/spark/SerializableWritable.scala | 3 + .../scala/org/apache/spark/SparkContext.scala | 86 ++++++++++++++++--- .../scala/org/apache/spark/SparkEnv.scala | 8 +- .../scala/org/apache/spark/TaskContext.scala | 6 ++ .../org/apache/spark/TaskEndReason.scala | 31 +++++-- .../spark/annotation/AlphaComponent.java | 26 ++++++ .../apache/spark/annotation/DeveloperApi.java | 30 +++++++ .../apache/spark/annotation/Experimental.java | 31 +++++++ .../apache/spark/api/java/JavaDoubleRDD.scala | 19 +++- .../apache/spark/api/java/JavaPairRDD.scala | 11 ++- .../apache/spark/api/java/JavaRDDLike.scala | 9 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../org/apache/spark/api/java/JavaUtils.scala | 2 +- .../spark/broadcast/BroadcastFactory.scala | 7 +- .../spark/executor/ExecutorSource.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 20 ++++- .../apache/spark/io/CompressionCodec.scala | 19 ++++ .../spark/metrics/sink/ConsoleSink.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 2 +- .../apache/spark/metrics/sink/JmxSink.scala | 2 +- .../spark/metrics/sink/MetricsServlet.scala | 2 +- .../org/apache/spark/metrics/sink/Sink.scala | 2 +- .../spark/metrics/source/JvmSource.scala | 2 +- .../apache/spark/metrics/source/Source.scala | 2 +- .../main/scala/org/apache/spark/package.scala | 9 ++ .../apache/spark/partial/BoundedDouble.scala | 6 +- .../apache/spark/partial/PartialResult.scala | 3 + .../apache/spark/rdd/AsyncRDDActions.scala | 3 + .../org/apache/spark/rdd/CoGroupedRDD.scala | 8 +- .../org/apache/spark/rdd/CoalescedRDD.scala | 4 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 13 ++- .../scala/org/apache/spark/rdd/EmptyRDD.scala | 4 +- .../org/apache/spark/rdd/HadoopRDD.scala | 6 ++ .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 6 ++ .../apache/spark/rdd/PairRDDFunctions.scala | 5 +- .../spark/rdd/PartitionPruningRDD.scala | 9 +- .../spark/rdd/PartitionwiseSampledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 29 ++++++- .../org/apache/spark/rdd/SampledRDD.scala | 2 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 + .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 + .../spark/rdd/ZippedPartitionsRDD.scala | 8 +- .../org/apache/spark/rdd/ZippedRDD.scala | 2 +- .../spark/scheduler/InputFormatInfo.scala | 3 + .../apache/spark/scheduler/JobLogger.scala | 4 +- .../apache/spark/scheduler/JobResult.scala | 10 ++- .../spark/scheduler/SparkListener.scala | 20 ++++- .../apache/spark/scheduler/SplitInfo.scala | 3 + .../apache/spark/scheduler/StageInfo.scala | 4 +- .../org/apache/spark/scheduler/TaskInfo.scala | 15 ++-- .../apache/spark/scheduler/TaskLocality.scala | 5 +- .../spark/serializer/JavaSerializer.scala | 7 ++ .../spark/serializer/KryoSerializer.scala | 4 + .../apache/spark/serializer/Serializer.scala | 15 +++- .../apache/spark/storage/BlockManager.scala | 8 +- .../apache/spark/storage/StorageUtils.scala | 2 + .../spark/util/BoundedPriorityQueue.scala | 2 +- .../apache/spark/util/CollectionsUtil.scala | 2 +- .../org/apache/spark/util/Distribution.scala | 4 +- .../org/apache/spark/util/FileLogger.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 4 + .../spark/util/TimeStampedHashSet.scala | 2 +- .../scala/org/apache/spark/util/Vector.scala | 1 + .../spark/util/collection/AppendOnlyMap.scala | 9 +- .../collection/ExternalAppendOnlyMap.scala | 6 +- .../spark/util/collection/OpenHashMap.scala | 5 +- .../spark/util/random/Pseudorandom.scala | 4 + .../spark/util/random/RandomSampler.scala | 8 ++ docs/_plugins/copy_api_dirs.rb | 24 +++--- docs/css/api-docs.css | 18 ++++ docs/js/api-docs.js | 26 ++++++ .../org/apache/spark/graphx/package.scala | 5 +- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../org/apache/spark/sql/SchemaRDD.scala | 21 ++--- .../spark/tools}/StoragePerfTester.scala | 4 +- 84 files changed, 614 insertions(+), 130 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/Experimental.java create mode 100644 docs/css/api-docs.css create mode 100644 docs/js/api-docs.js rename {core/src/main/scala/org/apache/spark/storage => tools/src/main/scala/org/apache/spark/tools}/StoragePerfTester.scala (97%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ceead59b79ed6..59fdf659c9e11 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,15 +17,18 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** + * :: DeveloperApi :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ +@DeveloperApi case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 1cd629c15bd46..2c31cc20211ff 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,19 +17,24 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** + * :: DeveloperApi :: * Base class for dependencies. */ +@DeveloperApi abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** + * :: DeveloperApi :: * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ +@DeveloperApi abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -41,6 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** + * :: DeveloperApi :: * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -48,6 +54,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ +@DeveloperApi class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -61,20 +68,24 @@ class ShuffleDependency[K, V]( /** + * :: DeveloperApi :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ +@DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** + * :: DeveloperApi :: * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD * @param outStart the start of the range in the child RDD * @param length the length of the range */ +@DeveloperApi class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 2eec09cd1c795..1e4dec86a0530 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,13 +21,16 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** + * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ +@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -84,9 +87,11 @@ trait FutureAction[T] extends Future[T] { /** + * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ +@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { @@ -148,10 +153,12 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** + * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ +@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index 9b1601d5b95fa..fd1802ba2f984 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -21,7 +21,7 @@ package org.apache.spark * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ -class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) +private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { def hasNext: Boolean = !context.interrupted && delegate.hasNext diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 7423082e34f47..e5e15617acb10 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,11 +21,18 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. + * + * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. + * This will likely be changed or removed in future releases. */ +@DeveloperApi trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index dff665cae6cb6..e50b9ac2291f9 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,6 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d7124616d3bfb..f7750514ae13d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.WholeTextFileInputFormat @@ -48,22 +49,35 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** + * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. - * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can - * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] - * from a list of input files or InputFormats for the application. */ -class SparkContext( - config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, - // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It - // contains a map from hostname to a list of input format splits on the host. - val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) - extends Logging { + +@DeveloperApi +class SparkContext(config: SparkConf) extends Logging { + + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. + private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + + /** + * :: DeveloperApi :: + * Alternative constructor for setting preferred locations where Spark will create executors. + * + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca + * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. + */ + @DeveloperApi + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData + } /** * Alternative constructor that allows setting common Spark properties directly @@ -93,10 +107,45 @@ class SparkContext( environment: Map[String, String] = Map(), preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { - this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), - preferredNodeLocationData) + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) + this.preferredNodeLocationData = preferredNodeLocationData } + // NOTE: The below constructors could be consolidated using default arguments. Due to + // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. + // Until we have a good workaround for that bug the constructors remain broken out. + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + */ + private[spark] def this(master: String, appName: String) = + this(master, appName, null, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + */ + private[spark] def this(master: String, appName: String, sparkHome: String) = + this(master, appName, sparkHome, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = + this(master, appName, sparkHome, jars, Map(), Map()) + private[spark] val conf = config.clone() /** @@ -189,7 +238,7 @@ class SparkContext( jars.foreach(addJar) } - def warnSparkMem(value: String): String = { + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value @@ -653,6 +702,9 @@ class SparkContext( def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = new UnionRDD(this, Seq(first) ++ rest) + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + // Methods for creating shared variables /** @@ -716,6 +768,11 @@ class SparkContext( postEnvironmentUpdate() } + /** + * :: DeveloperApi :: + * Register a listener to receive up-calls from events that happen during execution. + */ + @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1021,8 +1078,10 @@ class SparkContext( } /** + * :: DeveloperApi :: * Run a job that can return approximate results. */ + @DeveloperApi def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, @@ -1040,6 +1099,7 @@ class SparkContext( /** * Submit a job for execution and return a FutureJob holding the result. */ + @Experimental def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9ea123f174b95..915315ed74436 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,6 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -35,13 +36,18 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** + * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + * + * NOTE: This is not intended for external use. This is exposed for Shark and may be made private + * in a future release. */ -class SparkEnv private[spark] ( +@DeveloperApi +class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index be53ca2968cfb..dc5a19ecd738e 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,8 +19,14 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +/** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + */ +@DeveloperApi class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index f1a753b6ab8a9..a3074916d13e7 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,29 +17,35 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason +@DeveloperApi +sealed trait TaskEndReason -private[spark] case object Success extends TaskEndReason +@DeveloperApi +case object Success extends TaskEndReason -private[spark] +@DeveloperApi case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] case class FetchFailed( +@DeveloperApi +case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason -private[spark] case class ExceptionFailure( +@DeveloperApi +case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], @@ -47,21 +53,28 @@ private[spark] case class ExceptionFailure( extends TaskEndReason /** + * :: DeveloperApi :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -private[spark] case object TaskResultLost extends TaskEndReason +@DeveloperApi +case object TaskResultLost extends TaskEndReason -private[spark] case object TaskKilled extends TaskEndReason +@DeveloperApi +case object TaskKilled extends TaskEndReason /** + * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -private[spark] case object ExecutorLostFailure extends TaskEndReason +@DeveloperApi +case object ExecutorLostFailure extends TaskEndReason /** + * :: DeveloperApi :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -private[spark] case object UnknownReason extends TaskEndReason +@DeveloperApi +case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java new file mode 100644 index 0000000000000..af01fb7cfbd04 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -0,0 +1,26 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** A new component of Spark which may have unstable API's. */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface AlphaComponent {} diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java new file mode 100644 index 0000000000000..5d546e7a63985 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -0,0 +1,30 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * A lower-level, unstable API intended for developers. + * + * Developer API's might change or be removed in minor versions of Spark. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface DeveloperApi {} diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java new file mode 100644 index 0000000000000..306b1418d8d0a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -0,0 +1,31 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * An experimental user-facing API. + * + * Experimental API's might change or be removed in minor versions of Spark, or be adopted as + * first-class Spark API's. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Experimental {} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index f816bb43a5b44..537f410b0ca26 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -184,14 +185,26 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e6c5d85917678..a41c7dbda2afc 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -27,11 +27,12 @@ import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} @@ -201,16 +202,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout).map(mapAsJavaMap) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index ae577b500ccb4..725c423a53e35 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -27,6 +27,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} @@ -343,16 +344,20 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def count(): Long = rdd.count() /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a2855d4db1d2e..1e8242a2cbbce 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -89,7 +89,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment)) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map())) private[spark] val env = sc.env diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index ecbf18849ad48..22810cb1c662d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.java import com.google.common.base.Optional -object JavaUtils { +private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index c7f7c59cfb449..8c8ce9b1691ac 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,16 +16,19 @@ */ package org.apache.spark.broadcast -import org.apache.spark.SecurityManager +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** - * An interface for all the broadcast implementations in Spark (to allow + * :: DeveloperApi :: + * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ +@DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 127f5e90f3e1a..0ed52cfe9df61 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 88625e79a5c68..e4f02a4be0b97 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,8 +17,14 @@ package org.apache.spark.executor +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} +/** + * :: DeveloperApi :: + * Metrics tracked during the execution of a task. + */ +@DeveloperApi class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -77,11 +83,16 @@ class TaskMetrics extends Serializable { var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics +private[spark] object TaskMetrics { + def empty(): TaskMetrics = new TaskMetrics } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data read in a given task. + */ +@DeveloperApi class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -116,6 +127,11 @@ class ShuffleReadMetrics extends Serializable { var remoteBytesRead: Long = _ } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data written in a given task. + */ +@DeveloperApi class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 059e58824c39b..e1a5ee316bb69 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,11 +23,18 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. + * + * Note: The wire protocol for a codec is not guaranteed compatible across versions of Spark. + * This is intended for use as an internal compression utility within a single + * Spark application. */ +@DeveloperApi trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -52,8 +59,14 @@ private[spark] object CompressionCodec { /** + * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ +@DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -65,9 +78,15 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** + * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ +@DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 64eac73605388..05852f1f98993 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -25,7 +25,7 @@ import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class ConsoleSink(val property: Properties, val registry: MetricRegistry, +private[spark] class ConsoleSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 val CONSOLE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 544848d4150b6..42c1200926fea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -26,7 +26,7 @@ import com.codahale.metrics.{CsvReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class CsvSink(val property: Properties, val registry: MetricRegistry, +private[spark] class CsvSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CSV_KEY_PERIOD = "period" val CSV_KEY_UNIT = "unit" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 7f0a2fd16fa99..aeb4ad44a0647 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -27,7 +27,7 @@ import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GraphiteSink(val property: Properties, val registry: MetricRegistry, +private[spark] class GraphiteSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 val GRAPHITE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 3b5edd5c376f0..ed27234b4e760 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -22,7 +22,7 @@ import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} import org.apache.spark.SecurityManager -class JmxSink(val property: Properties, val registry: MetricRegistry, +private[spark] class JmxSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 854b52c510e3d..571539ba5e467 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -30,7 +30,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -class MetricsServlet(val property: Properties, val registry: MetricRegistry, +private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 3a739aa563eae..6f2b5a06027ea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -trait Sink { +private[spark] trait Sink { def start: Unit def stop: Unit } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index 75cb2b8973aa1..f865f9648a91e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -20,7 +20,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -class JvmSource extends Source { +private[spark] class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala index 3fee55cc6dcd5..1dda2cd83b2a9 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala @@ -19,7 +19,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry -trait Source { +private[spark] trait Source { def sourceName: String def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 2625a7f6a575a..59bbb1171f239 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -32,7 +32,16 @@ package org.apache * * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. + * + * Classes and methods marked with + * Experimental are user-facing features which have not been officially adopted by the + * Spark project. These are subject to change or removal in minor releases. + * + * Classes and methods marked with + * Developer API are intended for advanced users want to extend Spark through lower + * level interfaces. These are subject to changes or removal in minor releases. */ + package object spark { // For package docs only } diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 5f4450859cc9b..aed0353344427 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,9 +17,13 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + /** - * A Double with error bars on it. + * :: Experimental :: + * A Double value with error bars and associated confidence. */ +@Experimental class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index 812368e04ac0d..eade07fbcbe37 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,6 +17,9 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + +@Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index d1c74a5063510..aed951a40b40c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,11 +24,14 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} +import org.apache.spark.annotation.Experimental /** + * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ +@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9aa454a5c8b88..c6e79557f08a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -51,12 +52,17 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** + * :: DeveloperApi :: * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * + * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * instantiating this directly. + * @param rdds parent RDDs. - * @param part partitioner used to partition the shuffle output. + * @param part partitioner used to partition the shuffle output */ +@DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 4e82b51313bf0..44401a663440c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark._ * @param parentsIndices list of indices in the parent that have been coalesced into this partition * @param preferredLocation the preferred location for this partition */ -case class CoalescedRDDPartition( +private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], @@ -70,7 +70,7 @@ case class CoalescedRDDPartition( * @param maxPartitions number of desired partitions in the coalesced RDD * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ -class CoalescedRDD[T: ClassTag]( +private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index a7b6b3b5146ce..2306c9736b334 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import org.apache.spark.annotation.Experimental import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator @@ -63,14 +64,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { */ def sampleVariance(): Double = stats().sampleVariance - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new MeanEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new SumEvaluator(self.partitions.size, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index a84e5f9fd8ef8..a2d7e344cf1b2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -22,9 +22,9 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkContext, TaskContext} /** - * An RDD that is empty, i.e. has no element in it. + * An RDD that has no partitions and no elements. */ -class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 3af008bd72378..6811e1abb8b70 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -70,9 +71,13 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.hadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. @@ -84,6 +89,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ +@DeveloperApi class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 1b503743ac117..a76a070b5b863 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx } - +// TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** * An RDD that executes an SQL query on a JDBC connection and reads results. * For usage example, see test case JdbcRDDSuite. diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 461a749eac48b..2d8dfa5a1645a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -36,15 +37,20 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ +@DeveloperApi class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index a92a84b5342d1..343e4325c0ef0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -39,6 +39,7 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ +import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner @@ -201,9 +202,11 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def countByKey(): Map[K, Long] = self.map(_._1).countByValue() /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = { self.map(_._1).countByValueApprox(timeout, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index b0440ca7f32cf..f781a8d776f2a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,8 +20,10 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} +import org.apache.spark.annotation.DeveloperApi -class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { +private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) + extends Partition { override val index = idx } @@ -30,7 +32,7 @@ class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends * Represents a dependency between the PartitionPruningRDD and its parent. In this * case, the child RDD contains a subset of partitions of the parents'. */ -class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) +private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends NarrowDependency[T](rdd) { @transient @@ -45,11 +47,13 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo /** + * :: DeveloperApi :: * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ +@DeveloperApi class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -63,6 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( } +@DeveloperApi object PartitionPruningRDD { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index ce4c0d382baab..b4e3bb5d75e17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -42,7 +42,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * @tparam T input RDD item type * @tparam U sampled RDD item type */ -class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( +private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], @transient seed: Long = System.nanoTime) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 41ae0fec823e7..e441d4a40ccd2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -37,7 +37,7 @@ import org.apache.spark.util.Utils * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassTag]( +private[spark] class PipedRDD[T: ClassTag]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], 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 74fa2a4fcd401..3437b2cac19c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -86,22 +87,34 @@ abstract class RDD[T: ClassTag]( // Methods that should be implemented by subclasses of RDD // ======================================================================= - /** Implemented by subclasses to compute a given partition. */ + /** + * :: DeveloperApi :: + * Implemented by subclasses to compute a given partition. + */ + @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] /** + * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperApi protected def getPartitions: Array[Partition] /** + * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps - /** Optionally overridden by subclasses to specify placement preferences. */ + /** + * :: DeveloperApi :: + * Optionally overridden by subclasses to specify placement preferences. + */ + @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -518,9 +531,11 @@ abstract class RDD[T: ClassTag]( } /** + * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ + @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -792,9 +807,11 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L @@ -838,8 +855,10 @@ abstract class RDD[T: ClassTag]( } /** - * (Experimental) Approximate version of countByValue(). + * :: Experimental :: + * Approximate version of countByValue(). */ + @Experimental def countByValueApprox( timeout: Long, confidence: Double = 0.95 @@ -860,6 +879,7 @@ abstract class RDD[T: ClassTag]( } /** + * :: Experimental :: * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation @@ -867,6 +887,7 @@ abstract class RDD[T: ClassTag]( * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. */ + @Experimental def countApproxDistinct(relativeSD: Double = 0.05): Long = { val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 4ceea557f569c..b097c30f8c231 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -33,7 +33,7 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition } @deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") -class SampledRDD[T: ClassTag]( +private[spark] class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, frac: Double, diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 02660ea6a45c5..802b0bdfb2d59 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -28,12 +29,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** + * :: DeveloperApi :: * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. */ +@DeveloperApi class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index a447030752096..21c6e07d69f90 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -43,6 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } +@DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index b56643444aa40..f3d30f6c9b32f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -41,7 +41,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassTag]( +private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]], preservesPartitioning: Boolean = false) @@ -74,7 +74,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassTag]( } } -class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( +private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], @@ -94,7 +94,7 @@ class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( } } -class ZippedPartitionsRDD3 +private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], @@ -119,7 +119,7 @@ class ZippedPartitionsRDD3 } } -class ZippedPartitionsRDD4 +private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 2119e76f0e032..b8110ffc42f2d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -44,7 +44,7 @@ private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( } } -class ZippedRDD[T: ClassTag, U: ClassTag]( +private[spark] class ZippedRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1: RDD[T], var rdd2: RDD[U]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index b3f2cb346f7da..bac37bfdaa23f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,11 +27,14 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** + * :: DeveloperApi :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ +@DeveloperApi class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 7c5053998f1d6..713aebfa3ce00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,9 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics /** + * :: DeveloperApi :: * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -38,7 +40,7 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ - +@DeveloperApi @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 047bd27056120..4cd6cbe189aab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,11 +17,17 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult +@DeveloperApi +sealed trait JobResult -private[spark] case object JobSucceeded extends JobResult +@DeveloperApi +case object JobSucceeded extends JobResult +@DeveloperApi private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d42e67742a4f7..ced20350d5356 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,21 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} +@DeveloperApi sealed trait SparkListenerEvent +@DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -46,20 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +@DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent +@DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -67,8 +80,11 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Interface for listening to events from the Spark scheduler. + * :: DeveloperApi :: + * Interface for listening to events from the Spark scheduler. Note that this is an internal + * interface which might change in different Spark releases. */ +@DeveloperApi trait SparkListener { /** * Called when a stage completes successfully or fails, with information on the completed stage. @@ -128,8 +144,10 @@ trait SparkListener { } /** + * :: DeveloperApi :: * Simple SparkListener that logs a few summary statistics when each stage completes */ +@DeveloperApi class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 5b40a3eb29b30..b85eabd6bbdbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,8 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi + // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. +@DeveloperApi class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index eec409b182ac6..9f732f7191465 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,12 +17,14 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo /** + * :: DeveloperApi :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ -private[spark] +@DeveloperApi class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6183b125def99..4c62e4dc0bac8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,13 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Information about a running task attempt inside a TaskSet. */ -private[spark] +@DeveloperApi class TaskInfo( val taskId: Long, val index: Int, @@ -46,15 +49,15 @@ class TaskInfo( var serializedSize: Int = 0 - def markGettingResult(time: Long = System.currentTimeMillis) { + private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } - def markSuccessful(time: Long = System.currentTimeMillis) { + private[spark] def markSuccessful(time: Long = System.currentTimeMillis) { finishTime = time } - def markFailed(time: Long = System.currentTimeMillis) { + private[spark] def markFailed(time: Long = System.currentTimeMillis) { finishTime = time failed = true } @@ -83,11 +86,11 @@ class TaskInfo( def duration: Long = { if (!finished) { - throw new UnsupportedOperationException("duration() called on unfinished tasks") + throw new UnsupportedOperationException("duration() called on unfinished task") } else { finishTime - launchTime } } - def timeRunning(currentTime: Long): Long = currentTime - launchTime + private[spark] def timeRunning(currentTime: Long): Long = currentTime - launchTime } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 308edb12edd5c..eb920ab0c0b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,10 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality extends Enumeration { +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi +object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 18a68b05fa853..5e5883554fcc1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -94,8 +95,14 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** + * :: DeveloperApi :: * A Spark serializer that uses Java's built-in serialization. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ +@DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 926e71573be32..d1e8c3ef63622 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -33,6 +33,10 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 099143494b851..9f04dc6e427c0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -22,20 +22,27 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** + * :: DeveloperApi :: * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. * * Implementations of this trait should implement: + * * 1. a zero-arg constructor or a constructor that accepts a [[org.apache.spark.SparkConf]] * as parameter. If both constructors are defined, the latter takes precedence. * * 2. Java serialization interface. + * + * Note that serializers are not required to be wire-compatible across different versions of Spark. + * They are intended to be used to serialize/de-serialize data within a single Spark application. */ +@DeveloperApi trait Serializer { def newInstance(): SerializerInstance } @@ -49,8 +56,10 @@ object Serializer { /** + * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. */ +@DeveloperApi trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -81,8 +90,10 @@ trait SerializerInstance { /** + * :: DeveloperApi :: * A stream for writing serialized objects. */ +@DeveloperApi trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -98,8 +109,10 @@ trait SerializationStream { /** + * :: DeveloperApi :: * A stream for reading serialized objects. */ +@DeveloperApi trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b021564477c47..a2a729130091f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -35,11 +35,11 @@ import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -sealed trait Values +private[spark] sealed trait Values -case class ByteBufferValues(buffer: ByteBuffer) extends Values -case class IteratorValues(iterator: Iterator[Any]) extends Values -case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends Values +private[spark] case class IteratorValues(iterator: Iterator[Any]) extends Values +private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values private[spark] class BlockManager( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index ff6e84cf9819a..07255aa366a6d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,6 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils private[spark] @@ -47,6 +48,7 @@ class StorageStatus( } } +@DeveloperApi private[spark] class RDDInfo( val id: Int, diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index c3692f2fd929b..b9f4a5d720b93 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -28,7 +28,7 @@ import scala.collection.generic.Growable * class and modifies it such that only the top K elements are retained. * The top K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) +private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends Iterable[A] with Growable[A] with Serializable { private val underlying = new JPriorityQueue[A](maxSize, ord) diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala index db3db87e6618e..93235031f3ad5 100644 --- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala +++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala @@ -22,7 +22,7 @@ import java.util import scala.Array import scala.reflect._ -object CollectionsUtils { +private[spark] object CollectionsUtils { def makeBinarySearch[K <% Ordered[K] : ClassTag] : (Array[K], K) => Int = { classTag[K] match { case ClassTag.Float => diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 5b347555fe708..a465298c8c5ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -29,7 +29,7 @@ import scala.collection.immutable.IndexedSeq * * Assumes you are giving it a non-empty set of data */ -class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { +private[spark] class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) @@ -69,7 +69,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) } } -object Distribution { +private[spark] object Distribution { def apply(data: Traversable[Double]): Option[Distribution] = { if (data.size > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index a0c07e32fdc98..b5f2ec6831d26 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -36,7 +36,7 @@ import org.apache.spark.io.CompressionCodec * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ -class FileLogger( +private[spark] class FileLogger( logDir: String, conf: SparkConf = new SparkConf, outputBufferSize: Int = 8 * 1024, // 8 KB diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index a6b39247a54ca..74fa77b68de0b 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,13 +17,17 @@ package org.apache.spark.util +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ +@DeveloperApi case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 19bece86b36b4..7cd8f28b12dd6 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Set -class TimeStampedHashSet[A] extends Set[A] { +private[spark] class TimeStampedHashSet[A] extends Set[A] { val internalMap = new ConcurrentHashMap[A, Long]() def contains(key: A): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index dc4b8f253f259..3c8f94a416c65 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.apache.spark.util.random.XORShiftRandom +@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index b8c852b4ff5c7..025492b177a77 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,7 +19,10 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * @@ -29,9 +32,9 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, - V)] with Serializable { +@DeveloperApi +class AppendOnlyMap[K, V](initialCapacity: Int = 64) + extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index caa06d5b445b4..dd01ae821f705 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -27,10 +27,12 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** + * :: DeveloperApi :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -55,8 +57,8 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ - -private[spark] class ExternalAppendOnlyMap[K, V, C]( +@DeveloperApi +class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index c26f23d50024a..62f99f3981793 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,14 +19,17 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. */ -private[spark] +@DeveloperApi class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 98569143ee1e3..70f3dd62b9b19 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -17,9 +17,13 @@ package org.apache.spark.util.random +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A class with pseudorandom behavior. */ +@DeveloperApi trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 0f1fca4813ba9..37a6b04f5200f 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -22,7 +22,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -30,6 +33,7 @@ import cern.jet.random.engine.DRand * @tparam T item type * @tparam U sampled item type */ +@DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -40,6 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** + * :: DeveloperApi :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -47,6 +52,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ +@DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -67,11 +73,13 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** + * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ +@DeveloperApi class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2245bcbc70f1e..bbd56d2fd13bb 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -24,7 +24,9 @@ external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] sql_projects = ["catalyst", "core", "hive"] - projects = core_projects + external_projects.map { |project_name| "external/" + project_name } + projects = core_projects + projects = projects + external_projects.map { |project_name| "external/" + project_name } + projects = projects + sql_projects.map { |project_name| "sql/" + project_name } puts "Moving to project root and building scaladoc." curr_dir = pwd @@ -42,24 +44,22 @@ source = "../" + project_name + "/target/scala-2.10/api" dest = "api/" + project_name - puts "echo making directory " + dest + puts "making directory " + dest mkdir_p dest # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) - end - - sql_projects.each do |project_name| - source = "../sql/" + project_name + "/target/scala-2.10/api/" - dest = "api/sql/" + project_name - puts "echo making directory " + dest - mkdir_p dest + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Append custom CSS + css = File.readlines("./css/api-docs.css") + css_file = dest + "/lib/template.css" + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end # Build Epydoc for Python diff --git a/docs/css/api-docs.css b/docs/css/api-docs.css new file mode 100644 index 0000000000000..b2d1d7f869790 --- /dev/null +++ b/docs/css/api-docs.css @@ -0,0 +1,18 @@ +/* Dynamically injected style for the API docs */ + +.developer { + background-color: #44751E; +} + +.experimental { + background-color: #257080; +} + +.alphaComponent { + background-color: #bb0000; +} + +.badge { + font-family: Arial, san-serif; + float: right; +} diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js new file mode 100644 index 0000000000000..1414b6d0b81a1 --- /dev/null +++ b/docs/js/api-docs.js @@ -0,0 +1,26 @@ +/* Dynamically injected post-processing code for the API docs */ + +$(document).ready(function() { + var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); + addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "Alpha Component"); + addBadges(annotations, "DeveloperApi", ":: DeveloperApi ::", "Developer API"); + addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); +}); + +function addBadges(allAnnotations, name, tag, html) { + var annotations = allAnnotations.filter(":contains('" + name + "')") + var tags = $(".cmt:contains(" + tag + ")") + + // Remove identifier tags from comments + tags.each(function(index) { + var oldHTML = $(this).html(); + var newHTML = oldHTML.replace(tag, ""); + $(this).html(newHTML); + }); + + // Add badges to all containers + tags.prevAll("h4.signature") + .add(annotations.closest("div.fullcommenttop")) + .add(annotations.closest("div.fullcomment").prevAll("h4.signature")) + .prepend(html); +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 425a5164cad24..ff17edeaf8f16 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -19,7 +19,10 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet -/** GraphX is a graph processing framework built on top of Spark. */ +/** + * ALPHA COMPONENT + * GraphX is a graph processing framework built on top of Spark. + */ package object graphx { /** * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 0f06ea088e1a1..77b5429bad432 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -90,7 +90,7 @@ trait MutableRow extends Row { def setString(ordinal: Int, value: String) /** - * EXPERIMENTAL + * Experimental * * Returns a mutable string builder for the specified column. A given row should return the * result of any mutations made to the returned buffer next time getString is called for the same diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 3193787680d16..d3d4c56bafe41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkContext +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl @@ -32,14 +33,14 @@ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT - * + * :: AlphaComponent :: * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * * @groupname userf Spark SQL Functions * @groupname Ungrouped Support functions for language integrated queries. */ +@AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends Logging with dsl.ExpressionConversions @@ -63,12 +64,12 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * EXPERIMENTAL - * + * :: Experimental :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. */ + @Experimental implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index fc95781448569..16da7fd92bffe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -26,8 +27,7 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType /** - * ALPHA COMPONENT - * + * :: AlphaComponent :: * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions, * SchemaRDDs can be used in relational queries, as shown in the examples below. * @@ -90,6 +90,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType * @groupprio schema -1 * @groupname Ungrouped Base RDD Functions */ +@AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, @transient protected[spark] val logicalPlan: LogicalPlan) @@ -228,8 +229,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use * scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of * the column is not known at compile time, all attributes are converted to strings before @@ -241,18 +241,19 @@ class SchemaRDD( * * @group Query */ + @Experimental def where(dynamicUdf: (DynamicRow) => Boolean) = new SchemaRDD( sqlContext, Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Returns a sampled version of the underlying dataset. * * @group Query */ + @Experimental def sample( fraction: Double, withReplacement: Boolean = true, @@ -260,8 +261,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Applies the given Generator, or table generating function, to this relation. * * @param generator A table generating function. The API for such functions is likely to change @@ -277,6 +277,7 @@ class SchemaRDD( * * @group Query */ + @Experimental def generate( generator: Generator, join: Boolean = false, @@ -285,8 +286,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such @@ -294,6 +294,7 @@ class SchemaRDD( * * @group schema */ + @Experimental def insertInto(tableName: String, overwrite: Boolean = false) = new SchemaRDD( sqlContext, diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala rename to tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8cea302eb14c3..8e8c35615a711 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.tools import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong @@ -25,7 +25,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils /** - * Utility for micro-benchmarking shuffle write performance. + * Internal utility for micro-benchmarking shuffle write performance. * * Writes simulated shuffle output from several threads and records the observed throughput. */ From bde9cc11fee42a0a41ec52d5dc7fa0502ce94f77 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 9 Apr 2014 02:21:15 -0700 Subject: [PATCH 12/12] [SPARK-1357] [MLLIB] Annotate developer and experimental APIs Annotate developer and experimental APIs in MLlib. Author: Xiangrui Meng Closes #298 from mengxr/api and squashes the following commits: 13390e8 [Xiangrui Meng] Merge branch 'master' into api dc4cbb3 [Xiangrui Meng] mark distribute matrices experimental 6b9f8e2 [Xiangrui Meng] add Experimental annotation 8773d0d [Xiangrui Meng] add DeveloperApi annotation da31733 [Xiangrui Meng] update developer and experimental tags 555e0fe [Xiangrui Meng] Merge branch 'master' into api ef1a717 [Xiangrui Meng] mark some constructors private add default parameters to JavaDoc 00ffbcc [Xiangrui Meng] update tree API annotation 0b674fa [Xiangrui Meng] mark decision tree APIs 86b9e34 [Xiangrui Meng] one pass over APIs of GLMs, NaiveBayes, and ALS f21d862 [Xiangrui Meng] Merge branch 'master' into api 2b133d6 [Xiangrui Meng] intial annotation of developer and experimental apis --- .../mllib/api/python/PythonMLLibAPI.scala | 4 ++ .../classification/LogisticRegression.scala | 18 +++---- .../mllib/classification/NaiveBayes.scala | 29 +++++++---- .../spark/mllib/classification/SVM.scala | 21 ++++---- .../spark/mllib/clustering/KMeans.scala | 49 ++++++++++++++++--- .../apache/spark/mllib/linalg/Vectors.scala | 19 ++++--- .../linalg/distributed/CoordinateMatrix.scala | 4 ++ .../distributed/DistributedMatrix.scala | 2 - .../linalg/distributed/IndexedRowMatrix.scala | 11 ++++- .../mllib/linalg/distributed/RowMatrix.scala | 4 ++ .../spark/mllib/optimization/Gradient.scala | 13 +++++ .../mllib/optimization/GradientDescent.scala | 15 ++++-- .../spark/mllib/optimization/Optimizer.scala | 7 +++ .../spark/mllib/optimization/Updater.scala | 13 +++++ .../spark/mllib/recommendation/ALS.scala | 34 ++++++++----- .../MatrixFactorizationModel.scala | 9 ++-- .../GeneralizedLinearAlgorithm.scala | 7 ++- .../apache/spark/mllib/regression/Lasso.scala | 18 ++++--- .../mllib/regression/LinearRegression.scala | 16 +++--- .../mllib/regression/RidgeRegression.scala | 20 ++++---- .../spark/mllib/tree/DecisionTree.scala | 10 ++-- .../spark/mllib/tree/configuration/Algo.scala | 5 ++ .../tree/configuration/FeatureType.scala | 5 ++ .../tree/configuration/QuantileStrategy.scala | 5 ++ .../mllib/tree/configuration/Strategy.scala | 6 ++- .../spark/mllib/tree/impurity/Entropy.scala | 32 +++++++----- .../spark/mllib/tree/impurity/Gini.scala | 10 +++- .../spark/mllib/tree/impurity/Impurity.scala | 12 ++++- .../spark/mllib/tree/impurity/Variance.scala | 8 +++ .../apache/spark/mllib/tree/model/Bin.scala | 1 + .../mllib/tree/model/DecisionTreeModel.scala | 4 ++ .../spark/mllib/tree/model/Filter.scala | 2 +- .../tree/model/InformationGainStats.scala | 5 ++ .../apache/spark/mllib/tree/model/Node.scala | 4 ++ .../apache/spark/mllib/tree/model/Split.scala | 12 +++-- .../spark/mllib/util/DataValidators.scala | 8 +-- .../mllib/util/KMeansDataGenerator.scala | 5 +- .../mllib/util/LinearDataGenerator.scala | 4 ++ .../LogisticRegressionDataGenerator.scala | 5 +- .../spark/mllib/util/MFDataGenerator.scala | 8 +-- .../org/apache/spark/mllib/util/MLUtils.scala | 9 +++- .../spark/mllib/util/SVMDataGenerator.scala | 4 ++ 42 files changed, 355 insertions(+), 122 deletions(-) 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 2df5b0d02b699..ae27c57799873 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 @@ -19,6 +19,7 @@ package org.apache.spark.mllib.api.python import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ @@ -28,8 +29,11 @@ import org.apache.spark.mllib.regression._ import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * The Java stubs necessary for the Python mllib bindings. */ +@DeveloperApi class PythonMLLibAPI extends Serializable { private def deserializeDoubleVector(bytes: Array[Byte]): Array[Double] = { val packetLength = bytes.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 798f3a5c94740..4f9eaacf67fe4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -55,7 +55,7 @@ class LogisticRegressionModel( this } - override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept val score = 1.0/ (1.0 + math.exp(-margin)) @@ -71,27 +71,27 @@ class LogisticRegressionModel( * NOTE: Labels used in Logistic Regression should be {0, 1} */ class LogisticRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { - val gradient = new LogisticGradient() - val updater = new SimpleUpdater() + private val gradient = new LogisticGradient() + private val updater = new SimpleUpdater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - override val validators = List(DataValidators.classificationLabels) + override protected val validators = List(DataValidators.binaryLabelValidator) /** * Construct a LogisticRegression object with default parameters */ def this() = this(1.0, 100, 0.0, 1.0) - def createModel(weights: Vector, intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index e956185319a69..5a45f12f1aa12 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -27,11 +28,16 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** + * :: Experimental :: + * * Model for Naive Bayes Classifiers. * - * @param pi Log of class priors, whose dimension is C. - * @param theta Log of class conditional probabilities, whose dimension is CxD. + * @param labels list of labels + * @param pi log of class priors, whose dimension is C, number of labels + * @param theta log of class conditional probabilities, whose dimension is C-by-D, + * where D is number of features */ +@Experimental class NaiveBayesModel( val labels: Array[Double], val pi: Array[Double], @@ -40,14 +46,17 @@ class NaiveBayesModel( private val brzPi = new BDV[Double](pi) private val brzTheta = new BDM[Double](theta.length, theta(0).length) - var i = 0 - while (i < theta.length) { - var j = 0 - while (j < theta(i).length) { - brzTheta(i, j) = theta(i)(j) - j += 1 + { + // Need to put an extra pair of braces to prevent Scala treating `i` as a member. + var i = 0 + while (i < theta.length) { + var j = 0 + while (j < theta(i).length) { + brzTheta(i, j) = theta(i)(j) + j += 1 + } + i += 1 } - i += 1 } override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) @@ -65,7 +74,7 @@ class NaiveBayesModel( * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). */ -class NaiveBayes private (var lambda: Double) extends Serializable with Logging { +class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { def this() = this(1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index e31a08899f8bc..956654b1fe90a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -55,7 +55,9 @@ class SVMModel( this } - override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { @@ -70,28 +72,27 @@ class SVMModel( * NOTE: Labels used in SVM should be {0, 1}. */ class SVMWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[SVMModel] with Serializable { - val gradient = new HingeGradient() - val updater = new SquaredL2Updater() + private val gradient = new HingeGradient() + private val updater = new SquaredL2Updater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - - override val validators = List(DataValidators.classificationLabels) + override protected val validators = List(DataValidators.binaryLabelValidator) /** * Construct a SVM object with default parameters */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Vector, intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new SVMModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index a78503df3134d..8f565eb60a60f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -37,12 +38,17 @@ import org.apache.spark.util.random.XORShiftRandom * to it should be cached by the user. */ class KMeans private ( - var k: Int, - var maxIterations: Int, - var runs: Int, - var initializationMode: String, - var initializationSteps: Int, - var epsilon: Double) extends Serializable with Logging { + private var k: Int, + private var maxIterations: Int, + private var runs: Int, + private var initializationMode: String, + private var initializationSteps: Int, + private var epsilon: Double) extends Serializable with Logging { + + /** + * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, + * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4}. + */ def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ @@ -71,6 +77,8 @@ class KMeans private ( } /** + * :: Experimental :: + * * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. @@ -316,8 +324,8 @@ object KMeans { data: RDD[Vector], k: Int, maxIterations: Int, - runs: Int = 1, - initializationMode: String = K_MEANS_PARALLEL): KMeansModel = { + runs: Int, + initializationMode: String): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) .setRuns(runs) @@ -325,6 +333,27 @@ object KMeans { .run(data) } + /** + * Trains a k-means model using specified parameters and the default values for unspecified. + */ + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + } + + /** + * Trains a k-means model using specified parameters and the default values for unspecified. + */ + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + } + /** * Returns the index of the closest center to the given point, as well as the squared distance. */ @@ -369,6 +398,10 @@ object KMeans { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } + /** + * :: Experimental :: + */ + @Experimental def main(args: Array[String]) { if (args.length < 4) { println("Usage: KMeans []") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 2cea58cd3fd22..99a849f1c66b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -64,11 +64,13 @@ trait Vector extends Serializable { /** * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. + * We don't use the name `Vector` because Scala imports + * [[scala.collection.immutable.Vector]] by default. */ object Vectors { /** - * Creates a dense vector. + * Creates a dense vector from its values. */ @varargs def dense(firstValue: Double, otherValues: Double*): Vector = @@ -158,20 +160,21 @@ class DenseVector(val values: Array[Double]) extends Vector { /** * A sparse vector represented by an index array and an value array. * - * @param n size of the vector. + * @param size size of the vector. * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double]) extends Vector { - - override def size: Int = n +class SparseVector( + override val size: Int, + val indices: Array[Int], + val values: Array[Double]) extends Vector { override def toString: String = { - "(" + n + "," + indices.zip(values).mkString("[", "," ,"]") + ")" + "(" + size + "," + indices.zip(values).mkString("[", "," ,"]") + ")" } override def toArray: Array[Double] = { - val data = new Array[Double](n) + val data = new Array[Double](size) var i = 0 val nnz = indices.length while (i < nnz) { @@ -181,5 +184,5 @@ class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double data } - private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, n) + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 9194f657494b2..89d5c03d76c42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors @@ -32,6 +33,8 @@ import org.apache.spark.mllib.linalg.Vectors case class MatrixEntry(i: Long, j: Long, value: Double) /** + * :: Experimental :: + * * Represents a matrix in coordinate format. * * @param entries matrix entries @@ -40,6 +43,7 @@ case class MatrixEntry(i: Long, j: Long, value: Double) * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the max column index plus one. */ +@Experimental class CoordinateMatrix( val entries: RDD[MatrixEntry], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index 13f72a3c724ef..a0e26ce3bc465 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -19,8 +19,6 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.mllib.linalg.Matrix - /** * Represents a distributively stored matrix backed by one or more RDDs. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index e110f070bd7c1..24c123ab7eb51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -19,14 +19,22 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.SingularValueDecomposition -/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +/** + * :: Experimental :: + * + * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. + */ +@Experimental case class IndexedRow(index: Long, vector: Vector) /** + * :: Experimental :: + * * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with * indexed rows. * @@ -36,6 +44,7 @@ case class IndexedRow(index: Long, vector: Vector) * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. */ +@Experimental class IndexedRowMatrix( val rows: RDD[IndexedRow], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f59811f18a68f..8d32c1a6dbba1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -23,11 +23,14 @@ import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging /** + * :: Experimental :: + * * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] @@ -36,6 +39,7 @@ import org.apache.spark.Logging * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. */ +@Experimental class RowMatrix( val rows: RDD[Vector], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 20654284965ed..1176dc9dbc08d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -19,11 +19,15 @@ package org.apache.spark.mllib.optimization import breeze.linalg.{axpy => brzAxpy} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to compute the gradient for a loss function, given a single data point. */ +@DeveloperApi abstract class Gradient extends Serializable { /** * Compute the gradient and loss given the features of a single data point. @@ -51,9 +55,12 @@ abstract class Gradient extends Serializable { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a logistic loss function, as used in binary classification. * See also the documentation for the precise formulation. */ +@DeveloperApi class LogisticGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze @@ -92,11 +99,14 @@ class LogisticGradient extends Gradient { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/n ||A weights-y||^2 * See also the documentation for the precise formulation. */ +@DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze @@ -124,10 +134,13 @@ class LeastSquaresGradient extends Gradient { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * NOTE: This assumes that the labels are {0,1} */ +@DeveloperApi class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index d0777ffd63ff8..04267d967dcad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -19,18 +19,22 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{Vector => BV, DenseVector => BDV} +import breeze.linalg.{DenseVector => BDV} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -class GradientDescent(var gradient: Gradient, var updater: Updater) +@DeveloperApi +class GradientDescent(private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { private var stepSize: Double = 1.0 @@ -107,7 +111,12 @@ class GradientDescent(var gradient: Gradient, var updater: Updater) } -// Top-level method to run gradient descent. +/** + * :: DeveloperApi :: + * + * Top-level method to run gradient descent. + */ +@DeveloperApi object GradientDescent extends Logging { /** * Run stochastic gradient descent (SGD) in parallel using mini batches. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index f9ce908a5f3b0..0a313f3104b14 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -19,8 +19,15 @@ package org.apache.spark.mllib.optimization import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector +/** + * :: DeveloperApi :: + * + * Trait for optimization problem solvers. + */ +@DeveloperApi trait Optimizer extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 3b7754cd7ac28..e67816796c6b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -21,9 +21,12 @@ import scala.math._ import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -35,6 +38,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} * The updater is responsible to also perform the update coming from the * regularization term R(w) (if any regularization is used). */ +@DeveloperApi abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize, iteration number and @@ -59,9 +63,12 @@ abstract class Updater extends Serializable { } /** + * :: DeveloperApi :: + * * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ +@DeveloperApi class SimpleUpdater extends Updater { override def compute( weightsOld: Vector, @@ -78,6 +85,8 @@ class SimpleUpdater extends Updater { } /** + * :: DeveloperApi :: + * * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -95,6 +104,7 @@ class SimpleUpdater extends Updater { * * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ +@DeveloperApi class L1Updater extends Updater { override def compute( weightsOld: Vector, @@ -120,10 +130,13 @@ class L1Updater extends Updater { } /** + * :: DeveloperApi :: + * * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. */ +@DeveloperApi class SquaredL2Updater extends Updater { override def compute( weightsOld: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 3124fac326d22..60cbb1c1e1d86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -22,6 +22,10 @@ import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting +import com.esotericsoftware.kryo.Kryo +import org.jblas.{DoubleMatrix, SimpleBlas, Solve} + +import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext, SparkConf} import org.apache.spark.storage.StorageLevel @@ -29,10 +33,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ -import com.esotericsoftware.kryo.Kryo -import org.jblas.{DoubleMatrix, SimpleBlas, Solve} - - /** * Out-link information for a user or product block. This includes the original user/product IDs * of the elements within this block, and the list of destination blocks that each user or @@ -90,14 +90,19 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * preferences rather than explicit ratings given to items. */ class ALS private ( - var numBlocks: Int, - var rank: Int, - var iterations: Int, - var lambda: Double, - var implicitPrefs: Boolean, - var alpha: Double, - var seed: Long = System.nanoTime() + private var numBlocks: Int, + private var rank: Int, + private var iterations: Int, + private var lambda: Double, + private var implicitPrefs: Boolean, + private var alpha: Double, + private var seed: Long = System.nanoTime() ) extends Serializable with Logging { + + /** + * Constructs an ALS instance with default parameters: {numBlocks: -1, rank: 10, iterations: 10, + * lambda: 0.01, implicitPrefs: false, alpha: 1.0}. + */ def this() = this(-1, 10, 10, 0.01, false, 1.0) /** @@ -127,11 +132,18 @@ class ALS private ( this } + /** Sets whether to use implicit preference. Default: false. */ def setImplicitPrefs(implicitPrefs: Boolean): ALS = { this.implicitPrefs = implicitPrefs this } + /** + * :: Experimental :: + * + * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. + */ + @Experimental def setAlpha(alpha: Double): ALS = { this.alpha = alpha this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 443fc5de5bf04..e05224fc7caf2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.recommendation +import org.jblas._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.api.python.PythonMLLibAPI -import org.jblas._ -import org.apache.spark.api.java.JavaRDD - /** * Model representing the result of matrix factorization. @@ -68,6 +69,8 @@ class MatrixFactorizationModel( } /** + * :: DeveloperApi :: + * * Predict the rating of many users for many products. * This is a Java stub for python predictAll() * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 80dc0f12ff84f..c24f5afb99686 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ @@ -79,7 +80,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected val validators: Seq[RDD[LabeledPoint] => Boolean] = List() - val optimizer: Optimizer + /** The optimizer to solve the problem. */ + def optimizer: Optimizer /** Whether to add intercept (default: true). */ protected var addIntercept: Boolean = true @@ -100,8 +102,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } /** + * :: Experimental :: + * * Set if the algorithm should validate data before training. Default true. */ + @Experimental def setValidateData(validateData: Boolean): this.type = { this.validateData = validateData this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 25920d0dc976e..5f0812fd2e0eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -52,15 +52,16 @@ class LassoModel( * See also the documentation for the precise formulation. */ class LassoWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LassoModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new L1Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + private val gradient = new LeastSquaresGradient() + private val updater = new L1Updater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) @@ -69,7 +70,8 @@ class LassoWithSGD private ( super.setIntercept(false) /** - * Construct a Lasso object with default parameters + * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, + * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 9ed927994e795..228fa8db3e721 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -52,19 +52,21 @@ class LinearRegressionModel( * See also the documentation for the precise formulation. */ class LinearRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new SimpleUpdater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + private val gradient = new LeastSquaresGradient() + private val updater = new SimpleUpdater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setMiniBatchFraction(miniBatchFraction) /** - * Construct a LinearRegression object with default parameters + * Construct a LinearRegression object with default parameters: {stepSize: 1.0, + * numIterations: 100, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 1f17d2107f940..e702027c7c170 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -52,16 +52,17 @@ class RidgeRegressionModel( * See also the documentation for the precise formulation. */ class RidgeRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new SquaredL2Updater() + private val gradient = new LeastSquaresGradient() + private val updater = new SquaredL2Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) @@ -70,7 +71,8 @@ class RidgeRegressionWithSGD private ( super.setIntercept(false) /** - * Construct a RidgeRegression object with default parameters + * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, + * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index dee9594a9dd79..c8a966cd5f5a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree import scala.util.control.Breaks._ +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.regression.LabeledPoint @@ -33,13 +34,16 @@ import org.apache.spark.util.random.XORShiftRandom import org.apache.spark.mllib.linalg.{Vector, Vectors} /** + * :: Experimental :: + * * A class that implements a decision tree algorithm for classification and regression. It * supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. */ -class DecisionTree private(val strategy: Strategy) extends Serializable with Logging { +@Experimental +class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { /** * Method to train a decision tree model over an RDD @@ -1024,7 +1028,7 @@ object DecisionTree extends Serializable with Logging { } } - val usage = """ + private val usage = """ Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] @@ -1113,7 +1117,7 @@ object DecisionTree extends Serializable with Logging { * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is * the label, and the second element represents the feature values (an array of Double). */ - def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { + private def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.trim().split(",") val label = parts(0).toDouble diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 2dd1f0f27b8f5..017f84f3b9e8b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum to select the algorithm for the decision tree */ +@Experimental object Algo extends Enumeration { type Algo = Value val Classification, Regression = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index 09ee0586c58fa..c0254c32c2dce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum to describe whether a feature is "continuous" or "categorical" */ +@Experimental object FeatureType extends Enumeration { type FeatureType = Value val Continuous, Categorical = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index 2457a480c2a14..b3e8b224beeaa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum for selecting the quantile calculation strategy */ +@Experimental object QuantileStrategy extends Enumeration { type QuantileStrategy = Value val Sort, MinMax, ApproxHist = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index df565f3eb8859..482faaa9e7256 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** + * :: Experimental :: + * * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation @@ -34,10 +37,11 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. */ +@Experimental class Strategy ( val algo: Algo, val impurity: Impurity, val maxDepth: Int, val maxBins: Int = 100, val quantileCalculationStrategy: QuantileStrategy = Sort, - val categoricalFeaturesInfo: Map[Int,Int] = Map[Int,Int]()) extends Serializable + val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int]()) extends Serializable diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index b93995fcf9441..55c43f2fcf9c5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,31 +17,39 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. */ +@Experimental object Entropy extends Impurity { - def log2(x: Double) = scala.math.log(x) / scala.math.log(2) + private[tree] def log2(x: Double) = scala.math.log(x) / scala.math.log(2) /** + * :: DeveloperApi :: + * * entropy calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return entropy value */ - def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - -(f0 * log2(f0)) - (f1 * log2(f1)) - } - } + @DeveloperApi + override def calculate(c0: Double, c1: Double): Double = { + if (c0 == 0 || c1 == 0) { + 0 + } else { + val total = c0 + c1 + val f0 = c0 / total + val f1 = c1 / total + -(f0 * log2(f0)) - (f1 * log2(f1)) + } + } - def calculate(count: Double, sum: Double, sumSquares: Double): Double = + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c0407554a91b3..c923b8e8f4cf1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,19 +17,27 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. */ +@Experimental object Gini extends Impurity { /** + * :: DeveloperApi :: + * * Gini coefficient calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return Gini coefficient value */ + @DeveloperApi override def calculate(c0: Double, c1: Double): Double = { if (c0 == 0 || c1 == 0) { 0 @@ -41,6 +49,6 @@ object Gini extends Impurity { } } - def calculate(count: Double, sum: Double, sumSquares: Double): Double = + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index a4069063af2ad..f407796596c6c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -17,26 +17,36 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Trait for calculating information gain. */ +@Experimental trait Impurity extends Serializable { /** + * :: DeveloperApi :: + * * information calculation for binary classification * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return information value */ + @DeveloperApi def calculate(c0 : Double, c1 : Double): Double /** + * :: DeveloperApi :: + * * information calculation for regression * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value */ + @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index b74577dcec167..2c64644f4ed0f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,19 +17,27 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating variance during regression */ +@Experimental object Variance extends Impurity { override def calculate(c0: Double, c1: Double): Double = throw new UnsupportedOperationException("Variance.calculate") /** + * :: DeveloperApi :: + * * variance calculation * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { val squaredLoss = sumSquares - (sum * sum) / count squaredLoss / count diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index a57faa13745f7..2d71e1e366069 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -30,4 +30,5 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * @param featureType type of feature -- categorical or continuous * @param category categorical label value accepted in the bin */ +private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a6dca84a2ce09..0f76f4a049057 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -17,15 +17,19 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector /** + * :: Experimental :: + * * Model to store the decision tree parameters * @param topNode root node * @param algo algorithm type -- classification or regression */ +@Experimental class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala index ebc9595eafef3..2deaf4ae8dcab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala @@ -22,7 +22,7 @@ package org.apache.spark.mllib.tree.model * @param split split specifying the feature index, type and threshold * @param comparison integer specifying <,=,> */ -case class Filter(split: Split, comparison: Int) { +private[tree] case class Filter(split: Split, comparison: Int) { // Comparison -1,0,1 signifies <.=,> override def toString = " split = " + split + "comparison = " + comparison } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 99bf79cf12e45..d36b58e92ced6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -17,7 +17,11 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: + * * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity @@ -25,6 +29,7 @@ package org.apache.spark.mllib.tree.model * @param rightImpurity right node impurity * @param predict predicted value */ +@DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index aac3f9ce308f7..339972141498c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vector /** + * :: DeveloperApi :: + * * Node in a decision tree * @param id integer node id * @param predict predicted value at the node @@ -31,6 +34,7 @@ import org.apache.spark.mllib.linalg.Vector * @param rightNode right child * @param stats information gain stats */ +@DeveloperApi class Node ( val id: Int, val predict: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 4e64a81dda74e..8bbb343079b49 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -17,20 +17,24 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** + * :: DeveloperApi :: + * * Split applied to a feature * @param feature feature index * @param threshold threshold for continuous feature * @param featureType type of feature -- categorical or continuous * @param categories accepted values for categorical variables */ +@DeveloperApi case class Split( feature: Int, threshold: Double, featureType: FeatureType, - categories: List[Double]){ + categories: List[Double]) { override def toString = "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType + @@ -42,7 +46,7 @@ case class Split( * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyLowSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyLowSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MinValue, featureType, List()) /** @@ -50,7 +54,7 @@ class DummyLowSplit(feature: Int, featureType: FeatureType) * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyHighSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyHighSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) /** @@ -59,6 +63,6 @@ class DummyHighSplit(feature: Int, featureType: FeatureType) * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyCategoricalSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyCategoricalSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 8b55bce7c4bec..230c409e1be33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,23 +17,25 @@ package org.apache.spark.mllib.util +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * A collection of methods used to validate data before applying ML algorithms. */ +@DeveloperApi object DataValidators extends Logging { /** * Function to check if labels used for classification are either zero or one. * - * @param data - input data set that needs to be checked - * * @return True if labels are all zero or one, false otherwise. */ - val classificationLabels: RDD[LabeledPoint] => Boolean = { data => + val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() if (numInvalid != 0) { logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 9109189dff52f..e693d13703987 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -19,15 +19,18 @@ package org.apache.spark.mllib.util import scala.util.Random +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. */ - +@DeveloperApi object KMeansDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 81e4eda2a68c4..140ff92869176 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -22,16 +22,20 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. */ +@DeveloperApi object LinearDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 61498dcc2be00..ca06b9ad58538 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,16 +19,19 @@ package org.apache.spark.mllib.util import scala.util.Random +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors /** + * :: DeveloperApi :: + * * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ - +@DeveloperApi object LogisticRegressionDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 348aba1dea5b6..3bd86d6813375 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -21,10 +21,13 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * Generate RDD(s) containing data for Matrix Factorization. * * This method samples training entries according to the oversampling factor @@ -47,9 +50,8 @@ import org.apache.spark.rdd.RDD * test (Boolean) Whether to create testing RDD. * testSampFact (Double) Percentage of training data to use as test data. */ - -object MFDataGenerator{ - +@DeveloperApi +object MFDataGenerator { def main(args: Array[String]) { if (args.length < 2) { println("Usage: MFDataGenerator " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 83d1bd3fd57fe..7f9804deaf33f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.util import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint @@ -122,6 +123,8 @@ object MLUtils { loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) /** + * :: Experimental :: + * * Load labeled data from a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -131,6 +134,7 @@ object MLUtils { * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is * the label, and the second element represents the feature values (an array of Double). */ + @Experimental def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.split(',') @@ -141,6 +145,8 @@ object MLUtils { } /** + * :: Experimental :: + * * Save labeled data to a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -148,6 +154,7 @@ object MLUtils { * @param data An RDD of LabeledPoints containing data to be saved. * @param dir Directory to save the data. */ + @Experimental def saveLabeledData(data: RDD[LabeledPoint], dir: String) { val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) dataStr.saveAsTextFile(dir) @@ -165,7 +172,7 @@ object MLUtils { * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats( + private[mllib] def computeStats( data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long): (Double, Vector, Vector) = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index e300c3dbe1fe0..87a6f2a0c3976 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -21,15 +21,19 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ +@DeveloperApi object SVMDataGenerator { def main(args: Array[String]) {