diff --git a/.gitignore b/.gitignore index c67cffa1c4375..3b9086c7187dc 100644 --- a/.gitignore +++ b/.gitignore @@ -5,6 +5,7 @@ *.ipr *.iml *.iws +*.pyc .idea/ .idea_modules/ sbt/*.jar @@ -49,6 +50,8 @@ dependency-reduced-pom.xml checkpoint derby.log dist/ +dev/create-release/*txt +dev/create-release/*new spark-*-bin-*.tgz unit-tests.log /lib/ diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 98a93d1fcb2a3..4efebcaa350fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -134,6 +134,7 @@ private[spark] class AppClient( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) + master ! ExecutorStateChanged(appId, id, ExecutorState.RUNNING, None, None) listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 8ba6a01bbcb97..f4fedc6327ab9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -144,8 +144,6 @@ private[spark] class ExecutorRunner( Files.write(header, stderr, UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) - state = ExecutorState.RUNNING - worker ! ExecutorStateChanged(appId, execId, state, None, None) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 6e66ddbdef788..1f755db485812 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -24,12 +24,12 @@ import org.apache.spark.input.StreamFileInputFormat import org.apache.spark.{ Partition, SparkContext } private[spark] class BinaryFileRDD[T]( - sc: SparkContext, - inputFormatClass: Class[_ <: StreamFileInputFormat[T]], - keyClass: Class[String], - valueClass: Class[T], - @transient conf: Configuration, - minPartitions: Int) + sc: SparkContext, + inputFormatClass: Class[_ <: StreamFileInputFormat[T]], + keyClass: Class[String], + valueClass: Class[T], + @transient conf: Configuration, + minPartitions: Int) extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { override def getPartitions: Array[Partition] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala deleted file mode 100644 index 9e41b3d1e2d4f..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ /dev/null @@ -1,35 +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.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] class FilteredRDD[T: ClassTag]( - prev: RDD[T], - f: T => Boolean) - extends RDD[T](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override val partitioner = prev.partitioner // Since filter cannot change a partition's keys - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).filter(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala deleted file mode 100644 index d8f87d4e3690e..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ /dev/null @@ -1,34 +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.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class FlatMappedRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], - f: T => TraversableOnce[U]) - extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).flatMap(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala deleted file mode 100644 index 7c9023f62d3b6..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala +++ /dev/null @@ -1,35 +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.rdd - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => TraversableOnce[U]) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[Product2[K, V]].partitions - - override val partitioner = firstParent[Product2[K, V]].partitioner - - override def compute(split: Partition, context: TaskContext) = { - firstParent[Product2[K, V]].iterator(split, context).flatMap { case Product2(k, v) => - f(v).map(x => (k, x)) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala deleted file mode 100644 index f6463fa715a71..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.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.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) - extends RDD[Array[T]](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - Array(firstParent[T].iterator(split, context).toArray).iterator -} diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala deleted file mode 100644 index 8d7c288593665..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ /dev/null @@ -1,32 +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.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) - extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).map(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala deleted file mode 100644 index a60952eee5901..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ /dev/null @@ -1,33 +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.rdd - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[Product2[K, U]].partitions - - override val partitioner = firstParent[Product2[K, U]].partitioner - - override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { - firstParent[Product2[K, V]].iterator(split, context).map { pair => (pair._1, f(pair._2)) } - } -} 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 e78e57678852f..c43e1f2fe135e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -660,7 +660,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def mapValues[U](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) - new MappedValuesRDD(self, cleanF) + new MapPartitionsRDD[(K, U), (K, V)](self, + (context, pid, iter) => iter.map { case (k, v) => (k, cleanF(v)) }, + preservesPartitioning = true) } /** @@ -669,7 +671,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) - new FlatMappedValuesRDD(self, cleanF) + new MapPartitionsRDD[(K, U), (K, V)](self, + (context, pid, iter) => iter.flatMap { case (k, v) => + cleanF(v).map(x => (k, x)) + }, + preservesPartitioning = true) } /** 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 8dfd952298f30..0bd616ec24fcb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.util.{Properties, Random} +import java.util.Random import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer @@ -36,13 +36,12 @@ import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, BernoulliCellSampler, SamplingUtils} @@ -270,19 +269,30 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[U: ClassTag](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) + def map[U: ClassTag](f: T => U): RDD[U] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.map(cleanF)) + } /** * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = - new FlatMappedRDD(this, sc.clean(f)) + def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.flatMap(cleanF)) + } /** * Return a new RDD containing only the elements that satisfy a predicate. */ - def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f)) + def filter(f: T => Boolean): RDD[T] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[T, T]( + this, + (context, pid, iter) => iter.filter(cleanF), + preservesPartitioning = true) + } /** * Return a new RDD containing the distinct elements in this RDD. @@ -503,7 +513,9 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD created by coalescing all elements within each partition into an array. */ - def glom(): RDD[Array[T]] = new GlommedRDD(this) + def glom(): RDD[Array[T]] = { + new MapPartitionsRDD[Array[T], T](this, (context, pid, iter) => Iterator(iter.toArray)) + } /** * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala index ee91a368b76ea..3bcc7178a3d8b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -66,8 +66,9 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { val curMem = threadMemory(threadId) val freeMemory = maxMemory - threadMemory.values.sum - // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads - val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem) + // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads; + // don't let it be negative + val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveThreads) - curMem)) if (curMem < maxMemory / (2 * numActiveThreads)) { // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking; diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 83170f7c5a4ab..2499c11a65b0e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,6 +17,7 @@ package org.apache.spark.storage +import java.io.{InputStream, IOException} import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} @@ -289,17 +290,22 @@ final class ShuffleBlockFetcherIterator( } val iteratorTry: Try[Iterator[Any]] = result match { - case FailureFetchResult(_, e) => Failure(e) - case SuccessFetchResult(blockId, _, buf) => { - val is = blockManager.wrapForCompression(blockId, buf.createInputStream()) - val iter = serializer.newInstance().deserializeStream(is).asIterator - Success(CompletionIterator[Any, Iterator[Any]](iter, { - // Once the iterator is exhausted, release the buffer and set currentResult to null - // so we don't release it again in cleanup. - currentResult = null - buf.release() - })) - } + case FailureFetchResult(_, e) => + Failure(e) + case SuccessFetchResult(blockId, _, buf) => + // There is a chance that createInputStream can fail (e.g. fetching a local file that does + // not exist, SPARK-4085). In that case, we should propagate the right exception so + // the scheduler gets a FetchFailedException. + Try(buf.createInputStream()).map { is0 => + val is = blockManager.wrapForCompression(blockId, is0) + val iter = serializer.newInstance().deserializeStream(is).asIterator + CompletionIterator[Any, Iterator[Any]](iter, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + currentResult = null + buf.release() + }) + } } (result.blockId, iteratorTry) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index cc3592ee43a35..bac6fdbcdc976 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicInteger - import org.scalatest.BeforeAndAfterAll import org.apache.spark.network.TransportContext diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 5a133c0490444..58a96245a9b53 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} import org.apache.spark.util.MutablePair abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { @@ -263,6 +264,28 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex } } } + + test("[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file") { + val myConf = conf.clone().set("spark.test.noStageRetry", "false") + sc = new SparkContext("local", "test", myConf) + val rdd = sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_ + _) + rdd.count() + + // Delete one of the local shuffle blocks. + val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0)) + val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0)) + assert(hashFile.exists() || sortFile.exists()) + + if (hashFile.exists()) { + hashFile.delete() + } + if (sortFile.exists()) { + sortFile.delete() + } + + // This count should retry the execution of the previous stage and rerun shuffle. + rdd.count() + } } object ShuffleSuite { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 6d9be796144b6..46fcb80fa1845 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -24,10 +24,9 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.util.Utils - import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ +import org.apache.spark.util.Utils class RDDSuite extends FunSuite with SharedSparkContext { @@ -37,8 +36,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) - assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? - assert(dups.distinct.collect === dups.distinct().collect) + assert(dups.distinct().count === 4) // Can distinct and count be called without parentheses? + assert(dups.distinct().collect === dups.distinct().collect) assert(dups.distinct(2).collect === dups.distinct().collect) assert(nums.reduce(_ + _) === 10) assert(nums.fold(0)(_ + _) === 10) @@ -617,9 +616,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { for(seed <- 1 to 5) { val splits = data.randomSplit(Array(1.0, 2.0, 3.0), seed) assert(splits.size == 3, "wrong number of splits") - assert(splits.flatMap(_.collect).sorted.toList == data.collect.toList, + assert(splits.flatMap(_.collect()).sorted.toList == data.collect().toList, "incomplete or wrong split") - val s = splits.map(_.count) + val s = splits.map(_.count()) assert(math.abs(s(0) - 100) < 50) // std = 9.13 assert(math.abs(s(1) - 200) < 50) // std = 11.55 assert(math.abs(s(2) - 300) < 50) // std = 12.25 @@ -762,8 +761,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val rdd3 = rdd2.map(_ + 1) val rdd4 = new UnionRDD(sc, List(rdd1, rdd2, rdd3)) assert(rdd4.parent(0).isInstanceOf[ParallelCollectionRDD[_]]) - assert(rdd4.parent(1).isInstanceOf[FilteredRDD[_]]) - assert(rdd4.parent(2).isInstanceOf[MappedRDD[_, _]]) + assert(rdd4.parent[Int](1) === rdd2) + assert(rdd4.parent[Int](2) === rdd3) } test("getNarrowAncestors") { @@ -781,20 +780,18 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Simple dependency tree with a single branch assert(ancestors1.size === 0) assert(ancestors2.size === 2) - assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) - assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors2.count(_ === rdd1) === 1) + assert(ancestors2.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) assert(ancestors3.size === 5) - assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) - assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) - assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 4) // Any ancestors before the shuffle are not considered assert(ancestors4.size === 0) assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 0) assert(ancestors5.size === 3) assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) - assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0) - assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + assert(ancestors5.count(_ === rdd3) === 0) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 2) } test("getNarrowAncestors with multiple parents") { @@ -815,16 +812,16 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Simple dependency tree with multiple branches assert(ancestors6.size === 3) assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) - assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors6.count(_ === rdd2) === 1) assert(ancestors7.size === 5) assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) - assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) - assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors7.count(_ === rdd2) === 1) + assert(ancestors7.count(_ === rdd3) === 1) // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) assert(ancestors8.size === 7) - assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) - assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_ === rdd2) === 1) + assert(ancestors8.count(_ === rdd3) === 1) assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) assert(ancestors8.count(_ == rdd1) === 1) @@ -834,7 +831,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Any ancestors before the shuffle are not considered assert(ancestors9.size === 2) assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) - assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) } /** @@ -868,12 +864,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ancestors3 = rdd3.getNarrowAncestors val ancestors4 = rdd4.getNarrowAncestors assert(ancestors3.size === 4) - assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) - assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 4) assert(ancestors3.count(_ == rdd3) === 0) assert(ancestors4.size === 4) - assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) - assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 3) assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) assert(ancestors4.count(_ == rdd3) === 1) assert(ancestors4.count(_ == rdd4) === 0) @@ -881,8 +875,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Cycles that do not involve the root val ancestors5 = rdd5.getNarrowAncestors assert(ancestors5.size === 6) - assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) - assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 5) assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) assert(ancestors4.count(_ == rdd3) === 1) @@ -890,8 +883,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ancestors6 = rdd6.getNarrowAncestors assert(ancestors6.size === 12) assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) - assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) - assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 7) assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala index d31bc22ee74f7..e0e646f0a3652 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala @@ -159,7 +159,7 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { test("threads can block to get at least 1 / 2N memory") { // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps - // for a bit and releases 250 bytes, which should then be greanted to t2. Further requests + // for a bit and releases 250 bytes, which should then be granted to t2. Further requests // by t2 will return false right away because it now has 1 / 2N of the memory. val manager = new ShuffleMemoryManager(1000L) @@ -291,4 +291,19 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") } } + + test("threads should not be granted a negative size") { + val manager = new ShuffleMemoryManager(1000L) + manager.tryToAcquire(700L) + + val latch = new CountDownLatch(1) + startThread("t1") { + manager.tryToAcquire(300L) + latch.countDown() + } + latch.await() // Wait until `t1` calls `tryToAcquire` + + val granted = manager.tryToAcquire(300L) + assert(0 === granted, "granted is negative") + } } diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index 99c29ef9ff8b6..a3b78a3eac6d0 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,8 +26,6 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) START_COMMIT = os.environ.get("START_COMMIT", "37b100") END_COMMIT = os.environ.get("END_COMMIT", "3693ae") @@ -40,8 +38,6 @@ END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") # Verify provided arguments -if not JIRA_USERNAME: sys.exit("JIRA_USERNAME must be provided") -if not JIRA_PASSWORD: sys.exit("JIRA_PASSWORD must be provided") start_commit_line = get_one_line(START_COMMIT) end_commit_line = get_one_line(END_COMMIT) num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) @@ -60,14 +56,6 @@ sys.exit("Ok, exiting") print "==================================================================================\n" -# Setup JIRA and github clients. We use two JIRA clients, one with authentication -# and one without, because authentication is slow and required only when we query -# JIRA user details but not Spark issues -jira_options = { "server": JIRA_API_BASE } -jira_client = JIRA(options = jira_options) -jira_client_auth = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) -github_client = Github() - # Find all commits within this range print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) commits = get_one_line_commits(START_COMMIT, END_COMMIT) @@ -105,13 +93,17 @@ def print_indented(_list): if reverts: print "Reverts (%d)" % len(reverts); print_indented(reverts) if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) print "==================== Warning: the above commits will be ignored ==================\n" -response = raw_input("%d commits left to process. Ok to proceed? [y/N] " % len(filtered_commits)) -if response.lower() != "y": +response = raw_input("%d commits left to process. Ok to proceed? [Y/n] " % len(filtered_commits)) +if response.lower() != "y" and response: sys.exit("Ok, exiting.") # Keep track of warnings to tell the user at the end warnings = [] +# Mapping from the invalid author name to its associated JIRA issues +# E.g. andrewor14 -> set("SPARK-2413", "SPARK-3551", "SPARK-3471") +invalid_authors = {} + # Populate a map that groups issues and components by author # It takes the form: Author name -> { Contribution type -> Spark components } # For instance, @@ -127,16 +119,23 @@ def print_indented(_list): # } # author_info = {} +jira_options = { "server": JIRA_API_BASE } +jira_client = JIRA(options = jira_options) print "\n=========================== Compiling contributor list ===========================" for commit in filtered_commits: commit_hash = re.findall("^[a-z0-9]+", commit)[0] issues = re.findall("SPARK-[0-9]+", commit.upper()) - # Translate the author in case the github username is not an actual name - # Also guard against any special characters used in the name - # Note the JIRA client we use here must have authentication enabled author = get_author(commit_hash) - author = unidecode.unidecode(unicode(author, "UTF-8")) - author = translate_author(author, github_client, jira_client_auth, warnings) + author = unidecode.unidecode(unicode(author, "UTF-8")).strip() # guard against special characters + # If the author name is invalid, keep track of it along + # with all associated issues so we can translate it later + if is_valid_author(author): + author = capitalize_author(author) + else: + if author not in invalid_authors: + invalid_authors[author] = set() + for issue in issues: + invalid_authors[author].add(issue) date = get_date(commit_hash) # Parse components from the commit message, if any commit_components = find_components(commit, commit_hash) @@ -147,7 +146,7 @@ def populate(issue_type, components): author_info[author] = {} if issue_type not in author_info[author]: author_info[author][issue_type] = set() - for component in all_components: + for component in components: author_info[author][issue_type].add(component) # Find issues and components associated with this commit for issue in issues: @@ -168,7 +167,6 @@ def populate(issue_type, components): # Each line takes the format "Author name - semi-colon delimited contributions" # e.g. Andrew Or - Bug fixes in Windows, Core, and Web UI; improvements in Core # e.g. Tathagata Das - Bug fixes and new features in Streaming -contributors_file_name = "contributors.txt" contributors_file = open(contributors_file_name, "w") authors = author_info.keys() authors.sort() @@ -192,11 +190,23 @@ def populate(issue_type, components): # Do not use python's capitalize() on the whole string to preserve case assert contribution contribution = contribution[0].capitalize() + contribution[1:] + # If the author name is invalid, use an intermediate format that + # can be translated through translate-contributors.py later + # E.g. andrewor14/SPARK-3425/SPARK-1157/SPARK-6672 + if author in invalid_authors and invalid_authors[author]: + author = author + "/" + "/".join(invalid_authors[author]) line = "%s - %s" % (author, contribution) contributors_file.write(line + "\n") contributors_file.close() print "Contributors list is successfully written to %s!" % contributors_file_name +# Prompt the user to translate author names if necessary +if invalid_authors: + warnings.append("Found the following invalid authors:") + for a in invalid_authors: + warnings.append("\t%s" % a) + warnings.append("Please run './translate-contributors.py' to translate them.") + # Log any warnings encountered in the process if warnings: print "\n============ Warnings encountered while creating the contributor list ============" diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 0d6830b11dc73..76a10c32886d4 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -44,6 +44,9 @@ print "Install using 'sudo pip install unidecode'" sys.exit(-1) +# Contributors list file name +contributors_file_name = "contributors.txt" + # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] def get_author(commit_hash): @@ -69,7 +72,8 @@ def num_commits_in_range(start_hash, end_hash): "build": "build fixes", "improvement": "improvements", "new feature": "new features", - "documentation": "documentation" + "documentation": "documentation", + "test": "test" } # Maintain a mapping for translating component names when creating the release notes @@ -182,36 +186,3 @@ def capitalize_author(author): words = [w[0].capitalize() + w[1:] for w in words if w] return " ".join(words) -# Maintain a mapping of translated author names as a cache -translated_authors = {} - -# Format the given author in a format appropriate for the contributors list. -# If the author is not an actual name, search github and JIRA for potential -# replacements and log all candidates as a warning. -def translate_author(github_author, github_client, jira_client, warnings): - if is_valid_author(github_author): - return capitalize_author(github_author) - # If the translated author is already cached, just return it - if github_author in translated_authors: - return translated_authors[github_author] - # Otherwise, author name is not found, so we need to search for an alternative name - candidates = set() - github_name = get_github_name(github_author, github_client) - jira_name = get_jira_name(github_author, jira_client) - if is_valid_author(github_name): github_name = capitalize_author(github_name) - if is_valid_author(jira_name): jira_name = capitalize_author(jira_name) - if github_name: candidates.add(github_name) - if jira_name: candidates.add(jira_name) - # Only use the github name as a replacement automatically - # The JIRA name may not make sense because it can belong to someone else - if is_valid_author(github_name): - candidates_message = " (another candidate is %s)" % jira_name if jira_name else "" - warnings.append("Replacing github user %s with %s%s" % (github_author, github_name, candidates_message)) - translated_authors[github_name] = github_name - return translated_authors[github_name] - # No direct replacement, so return the original author and list any candidates found - candidates_message = " (candidates: %s)" % nice_join(candidates) if candidates else "" - warnings.append("Unable to find a replacement for github user %s%s" % (github_author, candidates_message)) - translated_authors[github_author] = github_author - return translated_authors[github_author] - diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py new file mode 100755 index 0000000000000..ef4625b003cb6 --- /dev/null +++ b/dev/create-release/translate-contributors.py @@ -0,0 +1,190 @@ +#!/usr/bin/env python +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# This script translates invalid authors in the contributors list generated +# by generate-contributors.py. When the script encounters an author name that +# is considered invalid, it searches Github and JIRA in an attempt to search +# for replacements. This tool runs in two modes: +# +# (1) Interactive mode: For each invalid author name, this script presents +# all candidate replacements to the user and awaits user response. In this +# mode, the user may also input a custom name. This is the default. +# +# (2) Non-interactive mode: For each invalid author name, this script replaces +# the name with the first valid candidate it can find. If there is none, it +# uses the original name. This can be enabled through the --non-interactive flag. + +import os +import sys + +from releaseutils import * + +# You must set the following before use! +JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) +if not JIRA_USERNAME or not JIRA_PASSWORD: + sys.exit("Both JIRA_USERNAME and JIRA_PASSWORD must be set") + +# Write new contributors list to .new +if not os.path.isfile(contributors_file_name): + print "Contributors file %s does not exist!" % contributors_file_name + print "Have you run ./generate-contributors.py yet?" + sys.exit(1) +contributors_file = open(contributors_file_name, "r") +new_contributors_file_name = contributors_file_name + ".new" +new_contributors_file = open(new_contributors_file_name, "w") +warnings = [] + +# In non-interactive mode, this script will choose the first replacement that is valid +INTERACTIVE_MODE = True +if len(sys.argv) > 1: + options = set(sys.argv[1:]) + if "--non-interactive" in options: + INTERACTIVE_MODE = False +if INTERACTIVE_MODE: + print "Running in interactive mode. To disable this, provide the --non-interactive flag." + +# Setup Github and JIRA clients +jira_options = { "server": JIRA_API_BASE } +jira_client = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) +github_client = Github() + +# Generate candidates for the given author. This should only be called if the given author +# name does not represent a full name as this operation is somewhat expensive. Under the +# hood, it makes several calls to the Github and JIRA API servers to find the candidates. +# +# This returns a list of (candidate name, source) 2-tuples. E.g. +# [ +# (NOT_FOUND, "No full name found for Github user andrewor14"), +# ("Andrew Or", "Full name of JIRA user andrewor14"), +# ("Andrew Orso", "Full name of SPARK-1444 assignee andrewor14"), +# ("Andrew Ordall", "Full name of SPARK-1663 assignee andrewor14"), +# (NOT_FOUND, "No assignee found for SPARK-1763") +# ] +NOT_FOUND = "Not found" +def generate_candidates(author, issues): + candidates = [] + # First check for full name of Github user + github_name = get_github_name(new_author, github_client) + if github_name: + candidates.append((github_name, "Full name of Github user %s" % new_author)) + else: + candidates.append((NOT_FOUND, "No full name found for Github user %s" % new_author)) + # Then do the same for JIRA user + jira_name = get_jira_name(new_author, jira_client) + if jira_name: + candidates.append((jira_name, "Full name of JIRA user %s" % new_author)) + else: + candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % new_author)) + # Then do the same for the assignee of each of the associated JIRAs + # Note that a given issue may not have an assignee, or the assignee may not have a full name + for issue in issues: + jira_issue = jira_client.issue(issue) + jira_assignee = jira_issue.fields.assignee + if jira_assignee: + user_name = jira_assignee.name + display_name = jira_assignee.displayName + if display_name: + candidates.append((display_name, "Full name of %s assignee %s" % (issue, user_name))) + else: + candidates.append((NOT_FOUND, "No full name found for %s assignee %" % (issue, user_name))) + else: + candidates.append((NOT_FOUND, "No assignee found for %s" % issue)) + # Guard against special characters in candidate names + # Note that the candidate name may already be in unicode (JIRA returns this) + for i, (candidate, source) in enumerate(candidates): + try: + candidate = unicode(candidate, "UTF-8") + except TypeError: + # already in unicode + pass + candidate = unidecode.unidecode(candidate).strip() + candidates[i] = (candidate, source) + return candidates + +# Translate each invalid author by searching for possible candidates from Github and JIRA +# In interactive mode, this script presents the user with a list of choices and have the user +# select from this list. Additionally, the user may also choose to enter a custom name. +# In non-interactive mode, this script picks the first valid author name from the candidates +# If no such name exists, the original name is used (without the JIRA numbers). +print "\n========================== Translating contributor list ==========================" +for line in contributors_file: + author = line.split(" - ")[0] + print "Processing author %s" % author + if not author: + print " ERROR: Expected the following format - " + print " ERROR: Actual = %s" % line + if not is_valid_author(author): + new_author = author.split("/")[0] + issues = author.split("/")[1:] + candidates = generate_candidates(new_author, issues) + # Print out potential replacement candidates along with the sources, e.g. + # [X] No full name found for Github user andrewor14 + # [0] Andrew Or - Full name of JIRA user andrewor14 + # [1] Andrew Orso - Full name of SPARK-1444 assignee andrewor14 + # [2] Andrew Ordall - Full name of SPARK-1663 assignee andrewor14 + # [X] No assignee found for SPARK-1763 + # [3] Custom + candidate_names = [] + for candidate, source in candidates: + if candidate == NOT_FOUND: + print " [X] %s" % source + else: + index = len(candidate_names) + candidate_names.append(candidate) + print " [%d] %s - %s" % (index, candidate, source) + custom_index = len(candidate_names) + # In interactive mode, additionally provide "custom" option and await user response + if INTERACTIVE_MODE: + print " [%d] Custom" % custom_index + response = raw_input(" Your choice: ") + while not response.isdigit() or int(response) > custom_index: + response = raw_input(" Please enter an integer between 0 and %d: " % custom_index) + response = int(response) + if response == custom_index: + new_author = raw_input(" Please type a custom name for this author: ") + else: + new_author = candidate_names[response] + # In non-interactive mode, just pick the first candidate + else: + valid_candidate_names = [name for name, _ in candidates\ + if is_valid_author(name) and name != NOT_FOUND] + if valid_candidate_names: + new_author = valid_candidate_names[0] + # Finally, capitalize the author and replace the original one with it + # If the final replacement is still invalid, log a warning + if is_valid_author(new_author): + new_author = capitalize_author(new_author) + else: + warnings.append("Unable to find a valid name %s for author %s" % (new_author, author)) + print " * Replacing %s with %s" % (author, new_author) + line = line.replace(author, new_author) + new_contributors_file.write(line) +print "==================================================================================\n" +contributors_file.close() +new_contributors_file.close() + +print "Translated contributors list successfully written to %s!" % new_contributors_file_name + +# Log any warnings encountered in the process +if warnings: + print "\n========== Warnings encountered while translating the contributor list ===========" + for w in warnings: print w + print "Please manually correct these in the final contributors list at %s." % new_contributors_file_name + print "==================================================================================\n" + diff --git a/docs/img/ml-Pipeline.png b/docs/img/ml-Pipeline.png new file mode 100644 index 0000000000000..607928906bedd Binary files /dev/null and b/docs/img/ml-Pipeline.png differ diff --git a/docs/img/ml-PipelineModel.png b/docs/img/ml-PipelineModel.png new file mode 100644 index 0000000000000..9ebc16719d365 Binary files /dev/null and b/docs/img/ml-PipelineModel.png differ diff --git a/docs/img/ml-Pipelines.pptx b/docs/img/ml-Pipelines.pptx new file mode 100644 index 0000000000000..1f773376abc7a Binary files /dev/null and b/docs/img/ml-Pipelines.pptx differ diff --git a/docs/ml-guide.md b/docs/ml-guide.md new file mode 100644 index 0000000000000..1c2e27341473b --- /dev/null +++ b/docs/ml-guide.md @@ -0,0 +1,702 @@ +--- +layout: global +title: Spark ML Programming Guide +--- + +Spark ML is Spark's new machine learning package. It is currently an alpha component but is potentially a successor to [MLlib](mllib-guide.html). The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. + +MLlib vs. Spark ML: + +* Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. Since Spark ML is an alpha component, its API may change in future releases. +* Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. See below for more details. +* Spark ML only has Scala and Java APIs, whereas MLlib also has a Python API. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Main Concepts + +Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple algorithms into a single pipeline, or workflow. This section covers the key concepts introduced by the Spark ML API. + +* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL as a dataset which can hold a variety of data types. +E.g., a dataset could have different columns storing text, feature vectors, true labels, and predictions. + +* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `SchemaRDD` into another `SchemaRDD`. +E.g., an ML model is a `Transformer` which transforms an RDD with features into an RDD with predictions. + +* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `SchemaRDD` to produce a `Transformer`. +E.g., a learning algorithm is an `Estimator` which trains on a dataset and produces a model. + +* **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. + +* **[`Param`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. + +## ML Dataset + +Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. +Spark ML adopts the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL in order to support a variety of data types under a unified Dataset concept. + +`SchemaRDD` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. +In addition to the types listed in the Spark SQL guide, `SchemaRDD` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types. + +A `SchemaRDD` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. + +Columns in a `SchemaRDD` are named. The code examples below use names such as "text," "features," and "label." + +## ML Algorithms + +### Transformers + +A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `SchemaRDD` into another, generally by appending one or more columns. +For example: + +* A feature transformer might take a dataset, read a column (e.g., text), convert it into a new column (e.g., feature vectors), append the new column to the dataset, and output the updated dataset. +* A learning model might take a dataset, read the column containing feature vectors, predict the label for each feature vector, append the labels as a new column, and output the updated dataset. + +### Estimators + +An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `SchemaRDD` and produces a `Transformer`. +For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling `fit()` trains a `LogisticRegressionModel`, which is a `Transformer`. + +### Properties of ML Algorithms + +`Transformer`s and `Estimator`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. + +Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). + +## Pipeline + +In machine learning, it is common to run a sequence of algorithms to process and learn from data. +E.g., a simple text document processing workflow might include several stages: + +* Split each document's text into words. +* Convert each document's words into a numerical feature vector. +* Learn a prediction model using the feature vectors and labels. + +Spark ML represents such a workflow as a [`Pipeline`](api/scala/index.html#org.apache.spark.ml.Pipeline), +which consists of a sequence of [`PipelineStage`s](api/scala/index.html#org.apache.spark.ml.PipelineStage) (`Transformer`s and `Estimator`s) to be run in a specific order. We will use this simple workflow as a running example in this section. + +### How It Works + +A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. +These stages are run in order, and the input dataset is modified as it passes through each stage. +For `Transformer` stages, the `transform()` method is called on the dataset. +For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the dataset. + +We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. + +

+ Spark ML Pipeline Example +

+ +Above, the top row represents a `Pipeline` with three stages. +The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red). +The bottom row represents data flowing through the pipeline, where cylinders indicate `SchemaRDD`s. +The `Pipeline.fit()` method is called on the original dataset which has raw text documents and labels. +The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words into the dataset. +The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the dataset. +Now, since `LogisticRegression` is an `Estimator`, the `Pipeline` first calls `LogisticRegression.fit()` to produce a `LogisticRegressionModel`. +If the `Pipeline` had more stages, it would call the `LogisticRegressionModel`'s `transform()` method on the dataset before passing the dataset to the next stage. + +A `Pipeline` is an `Estimator`. +Thus, after a `Pipeline`'s `fit()` method runs, it produces a `PipelineModel` which is a `Transformer`. This `PipelineModel` is used at *test time*; the figure below illustrates this usage. + +

+ Spark ML PipelineModel Example +

+ +In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. +When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed through the `Pipeline` in order. +Each stage's `transform()` method updates the dataset and passes it to the next stage. + +`Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. + +### Details + +*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. + +*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `SchemaRDD`. + +## Parameters + +Spark ML `Estimator`s and `Transformer`s use a uniform API for specifying parameters. + +A [`Param`](api/scala/index.html#org.apache.spark.ml.param.Param) is a named parameter with self-contained documentation. +A [`ParamMap`](api/scala/index.html#org.apache.spark.ml.param.ParamMap) is a set of (parameter, value) pairs. + +There are two main ways to pass parameters to an algorithm: + +1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. This API resembles the API used in MLlib. +2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. + +Parameters belong to specific instances of `Estimator`s and `Transformer`s. +For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. +This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. + +# Code Examples + +This section gives code examples illustrating the functionality discussed above. +There is not yet documentation for specific algorithms in Spark ML. For more info, please refer to the [API Documentation](api/scala/index.html#org.apache.spark.ml.package). Spark ML algorithms are currently wrappers for MLlib algorithms, and the [MLlib programming guide](mllib-guide.html) has details on specific algorithms. + +## Example: Estimator, Transformer, and Param + +This example covers the concepts of `Estimator`, `Transformer`, and `Param`. + +
+ +
+{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.sql.{Row, SQLContext} + +val conf = new SparkConf().setAppName("SimpleParamsExample") +val sc = new SparkContext(conf) +val sqlContext = new SQLContext(sc) +import sqlContext._ + +// Prepare training data. +// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes +// into SchemaRDDs, where it uses the case class metadata to infer the schema. +val training = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) + +// Create a LogisticRegression instance. This instance is an Estimator. +val lr = new LogisticRegression() +// Print out the parameters, documentation, and any default values. +println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") + +// We may set parameters using setter methods. +lr.setMaxIter(10) + .setRegParam(0.01) + +// Learn a LogisticRegression model. This uses the parameters stored in lr. +val model1 = lr.fit(training) +// Since model1 is a Model (i.e., a Transformer produced by an Estimator), +// we can view the parameters it used during fit(). +// This prints the parameter (name: value) pairs, where names are unique IDs for this +// LogisticRegression instance. +println("Model 1 was fit using parameters: " + model1.fittingParamMap) + +// We may alternatively specify parameters using a ParamMap, +// which supports several methods for specifying parameters. +val paramMap = ParamMap(lr.maxIter -> 20) +paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. +paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.5) // Specify multiple Params. + +// One can also combine ParamMaps. +val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Changes output column name. +val paramMapCombined = paramMap ++ paramMap2 + +// Now learn a new model using the paramMapCombined parameters. +// paramMapCombined overrides all parameters set earlier via lr.set* methods. +val model2 = lr.fit(training, paramMapCombined) +println("Model 2 was fit using parameters: " + model2.fittingParamMap) + +// Prepare test documents. +val test = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) + +// Make predictions on test documents using the Transformer.transform() method. +// LogisticRegression.transform will only use the 'features' column. +// Note that model2.transform() outputs a 'probability' column instead of the usual 'score' +// column since we renamed the lr.scoreCol parameter previously. +model2.transform(test) + .select('features, 'label, 'probability, 'prediction) + .collect() + .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => + println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) + } +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); +JavaSparkContext jsc = new JavaSparkContext(conf); +JavaSQLContext jsql = new JavaSQLContext(jsc); + +// Prepare training data. +// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes +// into SchemaRDDs, where it uses the case class metadata to infer the schema. +List localTraining = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); +JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + +// Create a LogisticRegression instance. This instance is an Estimator. +LogisticRegression lr = new LogisticRegression(); +// Print out the parameters, documentation, and any default values. +System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); + +// We may set parameters using setter methods. +lr.setMaxIter(10) + .setRegParam(0.01); + +// Learn a LogisticRegression model. This uses the parameters stored in lr. +LogisticRegressionModel model1 = lr.fit(training); +// Since model1 is a Model (i.e., a Transformer produced by an Estimator), +// we can view the parameters it used during fit(). +// This prints the parameter (name: value) pairs, where names are unique IDs for this +// LogisticRegression instance. +System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap()); + +// We may alternatively specify parameters using a ParamMap. +ParamMap paramMap = new ParamMap(); +paramMap.put(lr.maxIter(), 20); // Specify 1 Param. +paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. +paramMap.put(lr.regParam(), 0.1); + +// One can also combine ParamMaps. +ParamMap paramMap2 = new ParamMap(); +paramMap2.put(lr.scoreCol(), "probability"); // Changes output column name. +ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); + +// Now learn a new model using the paramMapCombined parameters. +// paramMapCombined overrides all parameters set earlier via lr.set* methods. +LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); +System.out.println("Model 2 was fit using parameters: " + model2.fittingParamMap()); + +// Prepare test documents. +List localTest = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); +JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + +// Make predictions on test documents using the Transformer.transform() method. +// LogisticRegression.transform will only use the 'features' column. +// Note that model2.transform() outputs a 'probability' column instead of the usual 'score' +// column since we renamed the lr.scoreCol parameter previously. +model2.transform(test).registerAsTable("results"); +JavaSchemaRDD results = + jsql.sql("SELECT features, label, probability, prediction FROM results"); +for (Row r: results.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + + ", prediction=" + r.get(3)); +} +{% endhighlight %} +
+ +
+ +## Example: Pipeline + +This example follows the simple text document `Pipeline` illustrated in the figures above. + +
+ +
+{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.sql.{Row, SQLContext} + +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from case classes. +case class LabeledDocument(id: Long, text: String, label: Double) +case class Document(id: Long, text: String) + +// Set up contexts. Import implicit conversions to SchemaRDD from sqlContext. +val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") +val sc = new SparkContext(conf) +val sqlContext = new SQLContext(sc) +import sqlContext._ + +// Prepare training documents, which are labeled. +val training = sparkContext.parallelize(Seq( + LabeledDocument(0L, "a b c d e spark", 1.0), + LabeledDocument(1L, "b d", 0.0), + LabeledDocument(2L, "spark f g h", 1.0), + LabeledDocument(3L, "hadoop mapreduce", 0.0))) + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") +val hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") +val lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01) +val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + +// Fit the pipeline to training documents. +val model = pipeline.fit(training) + +// Prepare test documents, which are unlabeled. +val test = sparkContext.parallelize(Seq( + Document(4L, "spark i j k"), + Document(5L, "l m n"), + Document(6L, "mapreduce spark"), + Document(7L, "apache hadoop"))) + +// Make predictions on test documents. +model.transform(test) + .select('id, 'text, 'score, 'prediction) + .collect() + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + } +{% endhighlight %} +
+ +
+{% highlight java %} +import java.io.Serializable; +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; +import org.apache.spark.SparkConf; + +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from Java Beans. +public class Document implements Serializable { + private Long id; + private String text; + + public Document(Long id, String text) { + this.id = id; + this.text = text; + } + + public Long getId() { return this.id; } + public void setId(Long id) { this.id = id; } + + public String getText() { return this.text; } + public void setText(String text) { this.text = text; } +} + +public class LabeledDocument extends Document implements Serializable { + private Double label; + + public LabeledDocument(Long id, String text, Double label) { + super(id, text); + this.label = label; + } + + public Double getLabel() { return this.label; } + public void setLabel(Double label) { this.label = label; } +} + +// Set up contexts. +SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); +JavaSparkContext jsc = new JavaSparkContext(conf); +JavaSQLContext jsql = new JavaSQLContext(jsc); + +// Prepare training documents, which are labeled. +List localTraining = Lists.newArrayList( + new LabeledDocument(0L, "a b c d e spark", 1.0), + new LabeledDocument(1L, "b d", 0.0), + new LabeledDocument(2L, "spark f g h", 1.0), + new LabeledDocument(3L, "hadoop mapreduce", 0.0)); +JavaSchemaRDD training = + jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); +HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); +LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + +// Fit the pipeline to training documents. +PipelineModel model = pipeline.fit(training); + +// Prepare test documents, which are unlabeled. +List localTest = Lists.newArrayList( + new Document(4L, "spark i j k"), + new Document(5L, "l m n"), + new Document(6L, "mapreduce spark"), + new Document(7L, "apache hadoop")); +JavaSchemaRDD test = + jsql.applySchema(jsc.parallelize(localTest), Document.class); + +// Make predictions on test documents. +model.transform(test).registerAsTable("prediction"); +JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); +for (Row r: predictions.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); +} +{% endhighlight %} +
+ +
+ +## Example: Model Selection via Cross-Validation + +An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. +`Pipeline`s facilitate model selection by making it easy to tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. + +Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) class, which takes an `Estimator`, a set of `ParamMap`s, and an [`Evaluator`](api/scala/index.html#org.apache.spark.ml.Evaluator). +`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets; e.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. +`CrossValidator` iterates through the set of `ParamMap`s. For each `ParamMap`, it trains the given `Estimator` and evaluates it using the given `Evaluator`. +The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. +`CrossValidator` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. + +The following example demonstrates using `CrossValidator` to select from a grid of parameters. +To help construct the parameter grid, we use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridBuilder) utility. + +Note that cross-validation over a grid of parameters is expensive. +E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. +In realistic settings, it can be common to try many more parameters and use more folds (`$k=3$` and `$k=10$` are common). +In other words, using `CrossValidator` can be very expensive. +However, it is also a well-established method for choosing parameters which is more statistically sound than heuristic hand-tuning. + +
+ +
+{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} +import org.apache.spark.sql.{Row, SQLContext} + +val conf = new SparkConf().setAppName("CrossValidatorExample") +val sc = new SparkContext(conf) +val sqlContext = new SQLContext(sc) +import sqlContext._ + +// Prepare training documents, which are labeled. +val training = sparkContext.parallelize(Seq( + LabeledDocument(0L, "a b c d e spark", 1.0), + LabeledDocument(1L, "b d", 0.0), + LabeledDocument(2L, "spark f g h", 1.0), + LabeledDocument(3L, "hadoop mapreduce", 0.0), + LabeledDocument(4L, "b spark who", 1.0), + LabeledDocument(5L, "g d a y", 0.0), + LabeledDocument(6L, "spark fly", 1.0), + LabeledDocument(7L, "was mapreduce", 0.0), + LabeledDocument(8L, "e spark program", 1.0), + LabeledDocument(9L, "a e c l", 0.0), + LabeledDocument(10L, "spark compile", 1.0), + LabeledDocument(11L, "hadoop software", 0.0))) + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") +val hashingTF = new HashingTF() + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") +val lr = new LogisticRegression() + .setMaxIter(10) +val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + +// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. +// This will allow us to jointly choose parameters for all Pipeline stages. +// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +val crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator) +// We use a ParamGridBuilder to construct a grid of parameters to search over. +// With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, +// this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. +val paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) + .addGrid(lr.regParam, Array(0.1, 0.01)) + .build() +crossval.setEstimatorParamMaps(paramGrid) +crossval.setNumFolds(2) // Use 3+ in practice + +// Run cross-validation, and choose the best set of parameters. +val cvModel = crossval.fit(training) +// Get the best LogisticRegression model (with the best set of parameters from paramGrid). +val lrModel = cvModel.bestModel + +// Prepare test documents, which are unlabeled. +val test = sparkContext.parallelize(Seq( + Document(4L, "spark i j k"), + Document(5L, "l m n"), + Document(6L, "mapreduce spark"), + Document(7L, "apache hadoop"))) + +// Make predictions on test documents. cvModel uses the best model found (lrModel). +cvModel.transform(test) + .select('id, 'text, 'score, 'prediction) + .collect() + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) +} +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.Model; +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.tuning.CrossValidator; +import org.apache.spark.ml.tuning.CrossValidatorModel; +import org.apache.spark.ml.tuning.ParamGridBuilder; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); +JavaSparkContext jsc = new JavaSparkContext(conf); +JavaSQLContext jsql = new JavaSQLContext(jsc); + +// Prepare training documents, which are labeled. +List localTraining = Lists.newArrayList( + new LabeledDocument(0L, "a b c d e spark", 1.0), + new LabeledDocument(1L, "b d", 0.0), + new LabeledDocument(2L, "spark f g h", 1.0), + new LabeledDocument(3L, "hadoop mapreduce", 0.0), + new LabeledDocument(4L, "b spark who", 1.0), + new LabeledDocument(5L, "g d a y", 0.0), + new LabeledDocument(6L, "spark fly", 1.0), + new LabeledDocument(7L, "was mapreduce", 0.0), + new LabeledDocument(8L, "e spark program", 1.0), + new LabeledDocument(9L, "a e c l", 0.0), + new LabeledDocument(10L, "spark compile", 1.0), + new LabeledDocument(11L, "hadoop software", 0.0)); +JavaSchemaRDD training = + jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); +HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); +LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + +// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. +// This will allow us to jointly choose parameters for all Pipeline stages. +// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +CrossValidator crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator()); +// We use a ParamGridBuilder to construct a grid of parameters to search over. +// With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, +// this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. +ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) + .addGrid(lr.regParam(), new double[]{0.1, 0.01}) + .build(); +crossval.setEstimatorParamMaps(paramGrid); +crossval.setNumFolds(2); // Use 3+ in practice + +// Run cross-validation, and choose the best set of parameters. +CrossValidatorModel cvModel = crossval.fit(training); +// Get the best LogisticRegression model (with the best set of parameters from paramGrid). +Model lrModel = cvModel.bestModel(); + +// Prepare test documents, which are unlabeled. +List localTest = Lists.newArrayList( + new Document(4L, "spark i j k"), + new Document(5L, "l m n"), + new Document(6L, "mapreduce spark"), + new Document(7L, "apache hadoop")); +JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + +// Make predictions on test documents. cvModel uses the best model found (lrModel). +cvModel.transform(test).registerAsTable("prediction"); +JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); +for (Row r: predictions.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); +} +{% endhighlight %} +
+ +
+ +# Dependencies + +Spark ML currently depends on MLlib and has the same dependencies. +Please see the [MLlib Dependencies guide](mllib-guide.html#Dependencies) for more info. + +Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies. + +# Developers + +**Development plan** + +If all goes well, `spark.ml` will become the primary ML package at the time of the Spark 1.3 release. Initially, simple wrappers will be used to port algorithms to `spark.ml`, but eventually, code will be moved to `spark.ml` and `spark.mllib` will be deprecated. + +**Advice to developers** + +During the next development cycle, new algorithms should be contributed to `spark.mllib`, but we welcome patches sent to either package. If an algorithm is best expressed using the new API (e.g., feature transformers), we may ask for developers to use the new `spark.ml` API. +Wrappers for old and new algorithms can be contributed to `spark.ml`. + +Users will be able to use algorithms from either of the two packages. The main difficulty will be the differences in APIs between the two packages. + diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 12a6afbeea829..fc8e732251a30 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Decision Tree and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, handle categorical features, extend to the multiclass classification setting, do not require -feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble +feature scaling, and are able to capture non-linearities and feature interactions. Tree ensemble algorithms such as random forests and boosting are among the top performers for classification and regression tasks. @@ -19,6 +19,8 @@ MLlib supports decision trees for binary and multiclass classification and for r using both continuous and categorical features. The implementation partitions data by rows, allowing distributed training with millions of instances. +Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described in the [Ensembles guide](mllib-ensembles.html). + ## Basic algorithm The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature @@ -42,18 +44,18 @@ impurity measure for regression (variance). Gini impurity Classification - $\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels. + $\sum_{i=1}^{C} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $C$ is the number of unique labels. Entropy Classification - $\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels. + $\sum_{i=1}^{C} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $C$ is the number of unique labels. Variance Regression - $\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, - $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$. + $\frac{1}{N} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, + $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^N x_i$. @@ -103,36 +105,73 @@ and the resulting `$M-1$` split candidates are considered. ### Stopping rule -The recursive tree construction is stopped at a node when one of the two conditions is met: +The recursive tree construction is stopped at a node when one of the following conditions is met: 1. The node depth is equal to the `maxDepth` training parameter. -2. No split candidate leads to an information gain at the node. +2. No split candidate leads to an information gain greater than `minInfoGain`. +3. No split candidate produces child nodes which each have at least `minInstancesPerNode` training instances. + +## Usage tips + +We include a few guidelines for using decision trees by discussing the various parameters. +The parameters are listed below roughly in order of descending importance. New users should mainly consider the "Problem specification parameters" section and the `maxDepth` parameter. + +### Problem specification parameters + +These parameters describe the problem you want to solve and your dataset. +They should be specified and do not require tuning. + +* **`algo`**: `Classification` or `Regression` + +* **`numClasses`**: Number of classes (for `Classification` only) + +* **`categoricalFeaturesInfo`**: Specifies which features are categorical and how many categorical values each of those features can take. This is given as a map from feature indices to feature arity (number of categories). Any features not in this map are treated as continuous. + * E.g., `Map(0 -> 2, 4 -> 10)` specifies that feature `0` is binary (taking values `0` or `1`) and that feature `4` has 10 categories (values `{0, 1, ..., 9}`). Note that feature indices are 0-based: features `0` and `4` are the 1st and 5th elements of an instance's feature vector. + * Note that you do not have to specify `categoricalFeaturesInfo`. The algorithm will still run and may get reasonable results. However, performance should be better if categorical features are properly designated. + +### Stopping criteria + +These parameters determine when the tree stops building (adding new nodes). +When tuning these parameters, be careful to validate on held-out test data to avoid overfitting. + +* **`maxDepth`**: Maximum depth of a tree. Deeper trees are more expressive (potentially allowing higher accuracy), but they are also more costly to train and are more likely to overfit. + +* **`minInstancesPerNode`**: For a node to be split further, each of its children must receive at least this number of training instances. This is commonly used with [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) since those are often trained deeper than individual trees. + +* **`minInfoGain`**: For a node to be split further, the split must improve at least this much (in terms of information gain). + +### Tunable parameters -## Implementation details +These parameters may be tuned. Be careful to validate on held-out test data when tuning in order to avoid overfitting. -### Max memory requirements +* **`maxBins`**: Number of bins used when discretizing continuous features. + * Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained split decisions. However, it also increases computation and communication. + * Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for any categorical feature. -For faster processing, the decision tree algorithm performs simultaneous histogram computations for -all nodes at each level of the tree. This could lead to high memory requirements at deeper levels -of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` -training parameter specifies the maximum amount of memory at the workers (twice as much at the -master) to be allocated to the histogram computation. The default value is conservatively chosen to -be 256 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements -for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each -subsequent level are split into smaller tasks. +* **`maxMemoryInMB`**: Amount of memory to be used for collecting sufficient statistics. + * The default value is conservatively chosen to be 256 MB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. + * *Implementation details*: For faster processing, the decision tree algorithm collects statistics about groups of nodes to split (rather than 1 node at a time). The number of nodes which can be handled in one group is determined by the memory requirements (which vary per features). The `maxMemoryInMB` parameter specifies the memory limit in terms of megabytes which each worker can use for these statistics. -Note that, if you have a large amount of memory, increasing `maxMemoryInMB` can lead to faster -training by requiring fewer passes over the data. +* **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. -### Binning feature values +* **`impurity`**: Impurity measure (discussed above) used to choose between candidate splits. This measure must match the `algo` parameter. -Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained -split decisions. However, it also increases computation and communication. +### Caching and checkpointing -Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for -any categorical feature. +MLlib 1.2 adds several features for scaling up to larger (deeper) trees and tree ensembles. When `maxDepth` is set to be large, it can be useful to turn on node ID caching and checkpointing. These parameters are also useful for [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) when `numTrees` is set to be large. -### Scaling +* **`useNodeIdCache`**: If this is set to true, the algorithm will avoid passing the current model (tree or trees) to executors on each iteration. + * This can be useful with deep trees (speeding up computation on workers) and for large Random Forests (reducing communication on each iteration). + * *Implementation details*: By default, the algorithm communicates the current model to executors so that executors can match training instances with tree nodes. When this setting is turned on, then the algorithm will instead cache this information. + +Node ID caching generates a sequence of RDDs (1 per iteration). This long lineage can cause performance problems, but checkpointing intermediate RDDs can alleviate those problems. +Note that checkpointing is only applicable when `useNodeIdCache` is set to true. + +* **`checkpointDir`**: Directory for checkpointing node ID cache RDDs. + +* **`checkpointInterval`**: Frequency for checkpointing node ID cache RDDs. Setting this too low will cause extra overhead from writing to HDFS; setting this too high can cause problems if executors fail and the RDD needs to be recomputed. + +## Scaling Computation scales approximately linearly in the number of training instances, in the number of features, and in the `maxBins` parameter. @@ -148,7 +187,7 @@ The example below demonstrates how to load a [LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), parse it as an RDD of `LabeledPoint` and then perform classification using a decision tree with Gini impurity as an impurity measure and a -maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. +maximum tree depth of 5. The test error is calculated to measure the algorithm accuracy.
@@ -158,8 +197,10 @@ import org.apache.spark.mllib.tree.DecisionTree import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. -// Cache the data since we will use it again to compute training error. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) // Train a DecisionTree model. // Empty categoricalFeaturesInfo indicates all features are continuous. @@ -169,17 +210,17 @@ val impurity = "gini" val maxDepth = 5 val maxBins = 32 -val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, - maxDepth, maxBins) +val model = DecisionTree.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins) -// Evaluate model on training instances and compute training error -val labelAndPreds = data.map { point => +// Evaluate model on test instances and compute test error +val labelAndPreds = testData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / data.count -println("Training Error = " + trainErr) -println("Learned classification tree model:\n" + model) +val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() +println("Test Error = " + testErr) +println("Learned classification tree model:\n" + model.toDebugString) {% endhighlight %}
@@ -187,7 +228,6 @@ println("Learned classification tree model:\n" + model) {% highlight java %} import java.util.HashMap; import scala.Tuple2; -import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -203,37 +243,42 @@ SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); JavaSparkContext sc = new JavaSparkContext(sparkConf); // Load and parse the data file. -// Cache the data since we will use it again to compute training error. String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; // Set parameters. // Empty categoricalFeaturesInfo indicates all features are continuous. Integer numClasses = 2; -HashMap categoricalFeaturesInfo = new HashMap(); +Map categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; Integer maxBins = 32; // Train a DecisionTree model for classification. -final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, +final DecisionTreeModel model = DecisionTree.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, impurity, maxDepth, maxBins); -// Evaluate model on training instances and compute training error +// Evaluate model on test instances and compute test error JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { return new Tuple2(model.predict(p.features()), p.label()); } }); -Double trainErr = +Double testErr = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override public Boolean call(Tuple2 pl) { + @Override + public Boolean call(Tuple2 pl) { return !pl._1().equals(pl._2()); } - }).count() / data.count(); -System.out.println("Training error: " + trainErr); -System.out.println("Learned classification tree model:\n" + model); + }).count() / testData.count(); +System.out.println("Test Error: " + testErr); +System.out.println("Learned classification tree model:\n" + model.toDebugString()); {% endhighlight %} @@ -244,26 +289,23 @@ from pyspark.mllib.tree import DecisionTree from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. -# Cache the data since we will use it again to compute training error. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. -model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, +model = DecisionTree.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, impurity='gini', maxDepth=5, maxBins=32) -# Evaluate model on training instances and compute training error -predictions = model.predict(data.map(lambda x: x.features)) -labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) -trainErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(data.count()) -print('Training Error = ' + str(trainErr)) +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) +print('Test Error = ' + str(testErr)) print('Learned classification tree model:') -print(model) +print(model.toDebugString()) {% endhighlight %} - -Note: When making predictions for a dataset, it is more efficient to do batch prediction rather -than separately calling `predict` on each data point. This is because the Python code makes calls -to an underlying `DecisionTree` model in Scala. @@ -285,8 +327,10 @@ import org.apache.spark.mllib.tree.DecisionTree import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. -// Cache the data since we will use it again to compute training error. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) // Train a DecisionTree model. // Empty categoricalFeaturesInfo indicates all features are continuous. @@ -295,17 +339,17 @@ val impurity = "variance" val maxDepth = 5 val maxBins = 32 -val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, +val model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo, impurity, maxDepth, maxBins) -// Evaluate model on training instances and compute training error -val labelsAndPredictions = data.map { point => +// Evaluate model on test instances and compute test error +val labelsAndPredictions = testData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("Training Mean Squared Error = " + trainMSE) -println("Learned regression tree model:\n" + model) +val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Test Mean Squared Error = " + testMSE) +println("Learned regression tree model:\n" + model.toDebugString) {% endhighlight %} @@ -325,45 +369,51 @@ import org.apache.spark.mllib.tree.model.DecisionTreeModel; import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.SparkConf; -// Load and parse the data file. -// Cache the data since we will use it again to compute training error. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); - SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); JavaSparkContext sc = new JavaSparkContext(sparkConf); +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + // Set parameters. // Empty categoricalFeaturesInfo indicates all features are continuous. -HashMap categoricalFeaturesInfo = new HashMap(); +Map categoricalFeaturesInfo = new HashMap(); String impurity = "variance"; Integer maxDepth = 5; Integer maxBins = 32; // Train a DecisionTree model. -final DecisionTreeModel model = DecisionTree.trainRegressor(data, +final DecisionTreeModel model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo, impurity, maxDepth, maxBins); -// Evaluate model on training instances and compute training error +// Evaluate model on test instances and compute test error JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { return new Tuple2(model.predict(p.features()), p.label()); } }); -Double trainMSE = +Double testMSE = predictionAndLabel.map(new Function, Double>() { - @Override public Double call(Tuple2 pl) { + @Override + public Double call(Tuple2 pl) { Double diff = pl._1() - pl._2(); return diff * diff; } }).reduce(new Function2() { - @Override public Double call(Double a, Double b) { + @Override + public Double call(Double a, Double b) { return a + b; } }) / data.count(); -System.out.println("Training Mean Squared Error: " + trainMSE); -System.out.println("Learned regression tree model:\n" + model); +System.out.println("Test Mean Squared Error: " + testMSE); +System.out.println("Learned regression tree model:\n" + model.toDebugString()); {% endhighlight %} @@ -374,26 +424,23 @@ from pyspark.mllib.tree import DecisionTree from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. -# Cache the data since we will use it again to compute training error. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. -model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, +model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo={}, impurity='variance', maxDepth=5, maxBins=32) -# Evaluate model on training instances and compute training error -predictions = model.predict(data.map(lambda x: x.features)) -labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) -trainMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(data.count()) -print('Training Mean Squared Error = ' + str(trainMSE)) +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) +print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression tree model:') -print(model) +print(model.toDebugString()) {% endhighlight %} - -Note: When making predictions for a dataset, it is more efficient to do batch prediction rather -than separately calling `predict` on each data point. This is because the Python code makes calls -to an underlying `DecisionTree` model in Scala. diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md new file mode 100644 index 0000000000000..23ede04b62d5b --- /dev/null +++ b/docs/mllib-ensembles.md @@ -0,0 +1,653 @@ +--- +layout: global +title: Ensembles - MLlib +displayTitle: MLlib - Ensembles +--- + +* Table of contents +{:toc} + +An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) +is a learning algorithm which creates a model composed of a set of other base models. +MLlib supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBosotedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest). +Both use [decision trees](mllib-decision-tree.html) as their base models. + +## Gradient-Boosted Trees vs. Random Forests + +Both [Gradient-Boosted Trees (GBTs)](mllib-ensembles.html#Gradient-Boosted-Trees-(GBTS)) and [Random Forests](mllib-ensembles.html#Random-Forests) are algorithms for learning ensembles of trees, but the training processes are different. There are several practical trade-offs: + + * GBTs train one tree at a time, so they can take longer to train than random forests. Random Forests can train multiple trees in parallel. + * On the other hand, it is often reasonable to use smaller (shallower) trees with GBTs than with Random Forests, and training smaller trees takes less time. + * Random Forests can be less prone to overfitting. Training more trees in a Random Forest reduces the likelihood of overfitting, but training more trees with GBTs increases the likelihood of overfitting. (In statistical language, Random Forests reduce variance by using more trees, whereas GBTs reduce bias by using more trees.) + * Random Forests can be easier to tune since performance improves monotonically with the number of trees (whereas performance can start to decrease for GBTs if the number of trees grows too large). + +In short, both algorithms can be effective, and the choice should be based on the particular dataset. + +## Random Forests + +[Random forests](http://en.wikipedia.org/wiki/Random_forest) +are ensembles of [decision trees](mllib-decision-tree.html). +Random forests are one of the most successful machine learning models for classification and +regression. They combine many decision trees in order to reduce the risk of overfitting. +Like decision trees, random forests handle categorical features, +extend to the multiclass classification setting, do not require +feature scaling, and are able to capture non-linearities and feature interactions. + +MLlib supports random forests for binary and multiclass classification and for regression, +using both continuous and categorical features. +MLlib implements random forests using the existing [decision tree](mllib-decision-tree.html) +implementation. Please see the decision tree guide for more information on trees. + +### Basic algorithm + +Random forests train a set of decision trees separately, so the training can be done in parallel. +The algorithm injects randomness into the training process so that each decision tree is a bit +different. Combining the predictions from each tree reduces the variance of the predictions, +improving the performance on test data. + +#### Training + +The randomness injected into the training process includes: + +* Subsampling the original dataset on each iteration to get a different training set (a.k.a. bootstrapping). +* Considering different random subsets of features to split on at each tree node. + +Apart from these randomizations, decision tree training is done in the same way as for individual decision trees. + +#### Prediction + +To make a prediction on a new instance, a random forest must aggregate the predictions from its set of decision trees. This aggregation is done differently for classification and regression. + +*Classification*: Majority vote. Each tree's prediction is counted as a vote for one class. The label is predicted to be the class which receives the most votes. + +*Regression*: Averaging. Each tree predicts a real value. The label is predicted to be the average of the tree predictions. + +### Usage tips + +We include a few guidelines for using random forests by discussing the various parameters. +We omit some decision tree parameters since those are covered in the [decision tree guide](mllib-decision-tree.html). + +The first two parameters we mention are the most important, and tuning them can often improve performance: + +* **`numTrees`**: Number of trees in the forest. + * Increasing the number of trees will decrease the variance in predictions, improving the model's test-time accuracy. + * Training time increases roughly linearly in the number of trees. + +* **`maxDepth`**: Maximum depth of each tree in the forest. + * Increasing the depth makes the model more expressive and powerful. However, deep trees take longer to train and are also more prone to overfitting. + * In general, it is acceptable to train deeper trees when using random forests than when using a single decision tree. One tree is more likely to overfit than a random forest (because of the variance reduction from averaging multiple trees in the forest). + +The next two parameters generally do not require tuning. However, they can be tuned to speed up training. + +* **`subsamplingRate`**: This parameter specifies the size of the dataset used for training each tree in the forest, as a fraction of the size of the original dataset. The default (1.0) is recommended, but decreasing this fraction can speed up training. + +* **`featureSubsetStrategy`**: Number of features to use as candidates for splitting at each tree node. The number is specified as a fraction or function of the total number of features. Decreasing this number will speed up training, but can sometimes impact performance if too low. + +### Examples + +#### Classification + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using a Random Forest. +The test error is calculated to measure the algorithm accuracy. + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a RandomForest model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val numTrees = 3 // Use more in practice. +val featureSubsetStrategy = "auto" // Let the algorithm choose. +val impurity = "gini" +val maxDepth = 4 +val maxBins = 32 + +val model = RandomForest.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) + +// Evaluate model on test instances and compute test error +val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() +println("Test Error = " + testErr) +println("Learned classification forest model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; +import java.util.HashMap; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestClassification"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Train a RandomForest model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Integer numClasses = 2; +HashMap categoricalFeaturesInfo = new HashMap(); +Integer numTrees = 3; // Use more in practice. +String featureSubsetStrategy = "auto"; // Let the algorithm choose. +String impurity = "gini"; +Integer maxDepth = 5; +Integer maxBins = 32; +Integer seed = 12345; + +final RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override + public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / testData.count(); +System.out.println("Test Error: " + testErr); +System.out.println("Learned classification forest model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.mllib.tree import RandomForest +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a RandomForest model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +# Note: Use larger numTrees in practice. +# Setting featureSubsetStrategy="auto" lets the algorithm choose. +model = RandomForest.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='gini', maxDepth=4, maxBins=32) + +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) +print('Test Error = ' + str(testErr)) +print('Learned classification forest model:') +print(model.toDebugString()) +{% endhighlight %} +
+ +
+ +#### Regression + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using a Random Forest. +The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a RandomForest model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val numTrees = 3 // Use more in practice. +val featureSubsetStrategy = "auto" // Let the algorithm choose. +val impurity = "variance" +val maxDepth = 4 +val maxBins = 32 + +val model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) + +// Evaluate model on test instances and compute test error +val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Test Mean Squared Error = " + testMSE) +println("Learned regression forest model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForest"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Map categoricalFeaturesInfo = new HashMap(); +String impurity = "variance"; +Integer maxDepth = 4; +Integer maxBins = 32; + +// Train a RandomForest model. +final RandomForestModel model = RandomForest.trainRegressor(trainingData, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testMSE = + predictionAndLabel.map(new Function, Double>() { + @Override + public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override + public Double call(Double a, Double b) { + return a + b; + } + }) / testData.count(); +System.out.println("Test Mean Squared Error: " + testMSE); +System.out.println("Learned regression forest model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.mllib.tree import RandomForest +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a RandomForest model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +# Note: Use larger numTrees in practice. +# Setting featureSubsetStrategy="auto" lets the algorithm choose. +model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='variance', maxDepth=4, maxBins=32) + +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) +print('Test Mean Squared Error = ' + str(testMSE)) +print('Learned regression forest model:') +print(model.toDebugString()) +{% endhighlight %} +
+ +
+ +## Gradient-Boosted Trees (GBTs) + +[Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting) +are ensembles of [decision trees](mllib-decision-tree.html). +GBTs iteratively train decision trees in order to minimize a loss function. +Like decision trees, GBTs handle categorical features, +extend to the multiclass classification setting, do not require +feature scaling, and are able to capture non-linearities and feature interactions. + +MLlib supports GBTs for binary classification and for regression, +using both continuous and categorical features. +MLlib implements GBTs using the existing [decision tree](mllib-decision-tree.html) implementation. Please see the decision tree guide for more information on trees. + +*Note*: GBTs do not yet support multiclass classification. For multiclass problems, please use +[decision trees](mllib-decision-tree.html) or [Random Forests](mllib-ensembles.html#Random-Forest). + +### Basic algorithm + +Gradient boosting iteratively trains a sequence of decision trees. +On each iteration, the algorithm uses the current ensemble to predict the label of each training instance and then compares the prediction with the true label. The dataset is re-labeled to put more emphasis on training instances with poor predictions. Thus, in the next iteration, the decision tree will help correct for previous mistakes. + +The specific mechanism for re-labeling instances is defined by a loss function (discussed below). With each iteration, GBTs further reduce this loss function on the training data. + +#### Losses + +The table below lists the losses currently supported by GBTs in MLlib. +Note that each loss is applicable to one of classification or regression, not both. + +Notation: $N$ = number of instances. $y_i$ = label of instance $i$. $x_i$ = features of instance $i$. $F(x_i)$ = model's predicted label for instance $i$. + + + + + + + + + + + + + + + + + + + + + + +
LossTaskFormulaDescription
Log LossClassification$2 \sum_{i=1}^{N} \log(1+\exp(-2 y_i F(x_i)))$Twice binomial negative log likelihood.
Squared ErrorRegression$\sum_{i=1}^{N} (y_i - F(x_i))^2$Also called L2 loss. Default loss for regression tasks.
Absolute ErrorRegression$\sum_{i=1}^{N} |y_i - F(x_i)|$Also called L1 loss. Can be more robust to outliers than Squared Error.
+ +### Usage tips + +We include a few guidelines for using GBTs by discussing the various parameters. +We omit some decision tree parameters since those are covered in the [decision tree guide](mllib-decision-tree.html). + +* **`loss`**: See the section above for information on losses and their applicability to tasks (classification vs. regression). Different losses can give significantly different results, depending on the dataset. + +* **`numIterations`**: This sets the number of trees in the ensemble. Each iteration produces one tree. Increasing this number makes the model more expressive, improving training data accuracy. However, test-time accuracy may suffer if this is too large. + +* **`learningRate`**: This parameter should not need to be tuned. If the algorithm behavior seems unstable, decreasing this value may improve stability. + +* **`algo`**: The algorithm or task (classification vs. regression) is set using the tree [Strategy] parameter. + + +### Examples + +GBTs currently have APIs in Scala and Java. Examples in both languages are shown below. + +#### Classification + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using Gradient-Boosted Trees with log loss. +The test error is calculated to measure the algorithm accuracy. + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.GradientBoostedTrees +import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a GradientBoostedTrees model. +// The defaultParams for Classification use LogLoss by default. +val boostingStrategy = BoostingStrategy.defaultParams("Classification") +boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. +boostingStrategy.treeStrategy.numClassesForClassification = 2 +boostingStrategy.treeStrategy.maxDepth = 5 +// Empty categoricalFeaturesInfo indicates all features are continuous. +boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() + +val model = GradientBoostedTrees.train(trainingData, boostingStrategy) + +// Evaluate model on test instances and compute test error +val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() +println("Test Error = " + testErr) +println("Learned classification GBT model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; +import java.util.HashMap; +import java.util.Map; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoostedTrees; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Train a GradientBoostedTrees model. +// The defaultParams for Classification use LogLoss by default. +BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Classification"); +boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. +boostingStrategy.getTreeStrategy().setNumClassesForClassification(2); +boostingStrategy.getTreeStrategy().setMaxDepth(5); +// Empty categoricalFeaturesInfo indicates all features are continuous. +Map categoricalFeaturesInfo = new HashMap(); +boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); + +final GradientBoostedTreesModel model = + GradientBoostedTrees.train(trainingData, boostingStrategy); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override + public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / testData.count(); +System.out.println("Test Error: " + testErr); +System.out.println("Learned classification GBT model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
+ +#### Regression + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using Gradient-Boosted Trees with Squared Error as the loss. +The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.GradientBoostedTrees +import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a GradientBoostedTrees model. +// The defaultParams for Regression use SquaredError by default. +val boostingStrategy = BoostingStrategy.defaultParams("Regression") +boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. +boostingStrategy.treeStrategy.maxDepth = 5 +// Empty categoricalFeaturesInfo indicates all features are continuous. +boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() + +val model = GradientBoostedTrees.train(trainingData, boostingStrategy) + +// Evaluate model on test instances and compute test error +val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Test Mean Squared Error = " + testMSE) +println("Learned regression GBT model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; +import java.util.HashMap; +import java.util.Map; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoostedTrees; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Train a GradientBoostedTrees model. +// The defaultParams for Regression use SquaredError by default. +BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Regression"); +boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. +boostingStrategy.getTreeStrategy().setMaxDepth(5); +// Empty categoricalFeaturesInfo indicates all features are continuous. +Map categoricalFeaturesInfo = new HashMap(); +boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); + +final GradientBoostedTreesModel model = + GradientBoostedTrees.train(trainingData, boostingStrategy); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testMSE = + predictionAndLabel.map(new Function, Double>() { + @Override + public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override + public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); +System.out.println("Test Mean Squared Error: " + testMSE); +System.out.println("Learned regression GBT model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 94fc98ce4fabe..efd7dda310712 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -1,6 +1,6 @@ --- layout: global -title: Machine Learning Library (MLlib) +title: Machine Learning Library (MLlib) Programming Guide --- MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities, @@ -16,8 +16,9 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * random data generation * [Classification and regression](mllib-classification-regression.html) * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) - * [decision trees](mllib-decision-tree.html) * [naive Bayes](mllib-naive-bayes.html) + * [decision trees](mllib-decision-tree.html) + * [ensembles of trees](mllib-ensembles.html) (Random Forests and Gradient-Boosted Trees) * [Collaborative filtering](mllib-collaborative-filtering.html) * alternating least squares (ALS) * [Clustering](mllib-clustering.html) @@ -34,6 +35,17 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. +# spark.ml: The New ML Package + +Spark 1.2 includes a new machine learning package called `spark.ml`, currently an alpha component but potentially a successor to `spark.mllib`. The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. + +See the **[spark.ml programming guide](ml-guide.html)** for more information on this package. + +Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. + +Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. +See the `spark.ml` programming guide linked above for more details. + # Dependencies MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), @@ -60,6 +72,32 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 # Migration Guide +## From 1.1 to 1.2 + +The only API changes in MLlib v1.2 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.2: + +1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number +of classes. In MLlib v1.1, this argument was called `numClasses` in Python and +`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. +This `numClasses` parameter is specified either via +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Breaking change)* The API for +[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. +This should generally not affect user code, unless the user manually constructs decision trees +(instead of using the `trainClassifier` or `trainRegressor` methods). +The tree `Node` now includes more information, including the probability of the predicted label +(for classification). + +3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. + +Examples in the Spark distribution and examples in the +[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. + ## From 1.0 to 1.1 The only API changes in MLlib v1.1 are in diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index bc914a1899801..44b7f67c57734 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -110,12 +110,16 @@ However, L1 regularization can help promote sparsity in weights leading to small It is not recommended to train models without any regularization, especially when the number of training examples is small. +### Optimization + +Under the hood, linear methods use convex optimization methods to optimize the objective functions. MLlib uses two methods, SGD and L-BFGS, described in the [optimization section](mllib-optimization.html). Currently, most algorithm APIs support Stochastic Gradient Descent (SGD), and a few support L-BFGS. Refer to [this optimization section](mllib-optimization.html#Choosing-an-Optimization-Method) for guidelines on choosing between optimization methods. + ## Binary classification [Binary classification](http://en.wikipedia.org/wiki/Binary_classification) aims to divide items into two categories: positive and negative. MLlib -supports two linear methods for binary classification: linear support vector -machines (SVMs) and logistic regression. For both methods, MLlib supports +supports two linear methods for binary classification: linear Support Vector +Machines (SVMs) and logistic regression. For both methods, MLlib supports L1 and L2 regularized variants. The training data set is represented by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical formulation in this guide, a training label $y$ is denoted as @@ -123,7 +127,7 @@ either $+1$ (positive) or $-1$ (negative), which is convenient for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, to be consistent with multiclass labeling. -### Linear support vector machines (SVMs) +### Linear Support Vector Machines (SVMs) The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) is a standard method for large-scale classification tasks. It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss function in the formulation given by the hinge loss: diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 45141c235be90..4d101afca2c97 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -138,6 +138,12 @@ vertical scalability issue (the number of training features) when computing the explicitly in Newton's method. As a result, L-BFGS often achieves rapider convergence compared with other first-order optimization. +### Choosing an Optimization Method + +[Linear methods](mllib-linear-methods.html) use optimization internally, and some linear methods in MLlib support both SGD and L-BFGS. +Different optimization methods can have different convergence guarantees depending on the properties of the objective function, and we cannot cover the literature here. +In general, when L-BFGS is available, we recommend using it instead of SGD since L-BFGS tends to converge faster (in fewer iterations). + ## Implementation in MLlib ### Gradient descent and stochastic gradient descent @@ -168,10 +174,7 @@ descent. All updaters in MLlib use a step size at the t-th step equal to * `regParam` is the regularization parameter when using L1 or L2 regularization. * `miniBatchFraction` is the fraction of the total data that is sampled in each iteration, to compute the gradient direction. - -Available algorithms for gradient descent: - -* [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) + * Sampling still requires a pass over the entire RDD, so decreasing `miniBatchFraction` may not speed up optimization much. Users will see the greatest speedup when the gradient is expensive to compute, for only the chosen samples are used for computing the gradient. ### L-BFGS L-BFGS is currently only a low-level optimization primitive in `MLlib`. If you want to use L-BFGS in various @@ -359,13 +362,15 @@ public class LBFGSExample { {% endhighlight %} -#### Developer's note + +## Developer's notes + Since the Hessian is constructed approximately from previous gradient evaluations, the objective function can not be changed during the optimization process. As a result, Stochastic L-BFGS will not work naively by just using miniBatch; therefore, we don't provide this until we have better understanding. -* `Updater` is a class originally designed for gradient decent which computes +`Updater` is a class originally designed for gradient decent which computes the actual gradient descent step. However, we're able to take the gradient and loss of objective function of regularization for L-BFGS by ignoring the part of logic only for gradient decent such as adaptive step size stuff. We will refactorize diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index dfe2db4b3fce8..e97ac9f0c4a00 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -30,7 +30,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.submit.file.replication - 3 + The default HDFS replication (usually 3) HDFS replication level for the files uploaded into HDFS for the application. These include things like the Spark jar, the app jar, and any distributed cache files/archives. @@ -91,6 +91,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). + + spark.yarn.queue + default + + The name of the YARN queue to which the application is submitted. + + spark.yarn.jar (none) @@ -132,6 +139,21 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The maximum number of threads to use in the application master for launching executor containers. + + spark.yarn.datanucleus.dir + $SPARK_HOME/lib + + The location of the DataNucleus jars, in case overriding the default location is desired. + By default, Spark on YARN will use the DataNucleus jars installed at + $SPARK_HOME/lib, but the jars can also be in a world-readable location on HDFS. + This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an + application runs. To point to a directory on HDFS, for example, set this configuration to + "hdfs:///some/path". + + This is required because the datanucleus jars cannot be packaged into the + assembly jar due to metadata conflicts (involving plugin.xml.) + + # Launching Spark on YARN diff --git a/docs/tuning.md b/docs/tuning.md index 9b5c9adac6a4f..0e2447dd46394 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -143,8 +143,7 @@ the space allocated to the RDD cache to mitigate this. **Measuring the Impact of GC** The first step in GC tuning is to collect statistics on how frequently garbage collection occurs and the amount of -time spent GC. This can be done by adding `-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps` to your -`SPARK_JAVA_OPTS` environment variable. Next time your Spark job is run, you will see messages printed in the worker's logs +time spent GC. This can be done by adding `-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps` to the Java options. (See the [configuration guide](configuration.html#Dynamically-Loading-Spark-Properties) for info on passing Java options to Spark jobs.) Next time your Spark job is run, you will see messages printed in the worker's logs each time a garbage collection occurs. Note these logs will be on your cluster's worker nodes (in the `stdout` files in their work directories), *not* on your driver program. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java new file mode 100644 index 0000000000000..f4b4f8d8c7b2f --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -0,0 +1,126 @@ +/* + * 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.ml; + +import java.util.List; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.tuning.CrossValidator; +import org.apache.spark.ml.tuning.CrossValidatorModel; +import org.apache.spark.ml.tuning.ParamGridBuilder; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +/** + * A simple example demonstrating model selection using CrossValidator. + * This example also demonstrates how Pipelines are Estimators. + * + * This example uses the Java bean classes {@link org.apache.spark.examples.ml.LabeledDocument} and + * {@link org.apache.spark.examples.ml.Document} defined in the Scala example + * {@link org.apache.spark.examples.ml.SimpleTextClassificationPipeline}. + * + * Run with + *
+ * bin/run-example ml.JavaCrossValidatorExample
+ * 
+ */ +public class JavaCrossValidatorExample { + + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + JavaSQLContext jsql = new JavaSQLContext(jsc); + + // Prepare training documents, which are labeled. + List localTraining = Lists.newArrayList( + new LabeledDocument(0L, "a b c d e spark", 1.0), + new LabeledDocument(1L, "b d", 0.0), + new LabeledDocument(2L, "spark f g h", 1.0), + new LabeledDocument(3L, "hadoop mapreduce", 0.0), + new LabeledDocument(4L, "b spark who", 1.0), + new LabeledDocument(5L, "g d a y", 0.0), + new LabeledDocument(6L, "spark fly", 1.0), + new LabeledDocument(7L, "was mapreduce", 0.0), + new LabeledDocument(8L, "e spark program", 1.0), + new LabeledDocument(9L, "a e c l", 0.0), + new LabeledDocument(10L, "spark compile", 1.0), + new LabeledDocument(11L, "hadoop software", 0.0)); + JavaSchemaRDD training = + jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); + HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + + // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. + // This will allow us to jointly choose parameters for all Pipeline stages. + // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + CrossValidator crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator()); + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, + // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. + ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) + .addGrid(lr.regParam(), new double[]{0.1, 0.01}) + .build(); + crossval.setEstimatorParamMaps(paramGrid); + crossval.setNumFolds(2); // Use 3+ in practice + + // Run cross-validation, and choose the best set of parameters. + CrossValidatorModel cvModel = crossval.fit(training); + + // Prepare test documents, which are unlabeled. + List localTest = Lists.newArrayList( + new Document(4L, "spark i j k"), + new Document(5L, "l m n"), + new Document(6L, "mapreduce spark"), + new Document(7L, "apache hadoop")); + JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + cvModel.transform(test).registerAsTable("prediction"); + JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); + for (Row r: predictions.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); + } + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java new file mode 100644 index 0000000000000..e25b271777ed4 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -0,0 +1,111 @@ +/* + * 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.ml; + +import java.util.List; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +/** + * A simple example demonstrating ways to specify parameters for Estimators and Transformers. + * Run with + * {{{ + * bin/run-example ml.JavaSimpleParamsExample + * }}} + */ +public class JavaSimpleParamsExample { + + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + JavaSQLContext jsql = new JavaSQLContext(jsc); + + // Prepare training data. + // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans + // into SchemaRDDs, where it uses the bean metadata to infer the schema. + List localTraining = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); + JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + + // Create a LogisticRegression instance. This instance is an Estimator. + LogisticRegression lr = new LogisticRegression(); + // Print out the parameters, documentation, and any default values. + System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); + + // We may set parameters using setter methods. + lr.setMaxIter(10) + .setRegParam(0.01); + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + LogisticRegressionModel model1 = lr.fit(training); + // Since model1 is a Model (i.e., a Transformer produced by an Estimator), + // we can view the parameters it used during fit(). + // This prints the parameter (name: value) pairs, where names are unique IDs for this + // LogisticRegression instance. + System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap()); + + // We may alternatively specify parameters using a ParamMap. + ParamMap paramMap = new ParamMap(); + paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. + paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. + paramMap.put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. + + // One can also combine ParamMaps. + ParamMap paramMap2 = new ParamMap(); + paramMap2.put(lr.scoreCol().w("probability")); // Change output column name + ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); + + // Now learn a new model using the paramMapCombined parameters. + // paramMapCombined overrides all parameters set earlier via lr.set* methods. + LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); + System.out.println("Model 2 was fit using parameters: " + model2.fittingParamMap()); + + // Prepare test documents. + List localTest = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); + JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + + // Make predictions on test documents using the Transformer.transform() method. + // LogisticRegression.transform will only use the 'features' column. + // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' + // column since we renamed the lr.scoreCol parameter previously. + model2.transform(test).registerAsTable("results"); + JavaSchemaRDD results = + jsql.sql("SELECT features, label, probability, prediction FROM results"); + for (Row r: results.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + + ", prediction=" + r.get(3)); + } + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index 22ba68d8c354c..54f18014e4b2f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -80,14 +80,14 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); - JavaSchemaRDD test = - jsql.applySchema(jsc.parallelize(localTest), Document.class); + JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. model.transform(test).registerAsTable("prediction"); JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); for (Row r: predictions.collect()) { - System.out.println(r); + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); } } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java index 4a5ac404ea5ea..a1844d5d07ad4 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java @@ -73,7 +73,7 @@ public static void main(String[] args) { return p.label(); } }).countByValue().size(); - boostingStrategy.treeStrategy().setNumClassesForClassification(numClasses); + boostingStrategy.treeStrategy().setNumClasses(numClasses); // Train a GradientBoosting model for classification. final GradientBoostedTreesModel model = GradientBoostedTrees.train(data, boostingStrategy); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java new file mode 100644 index 0000000000000..89a4e092a5af7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java @@ -0,0 +1,139 @@ +/* + * 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 scala.Tuple2; + +import java.util.HashMap; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; + +public final class JavaRandomForestExample { + + /** + * Note: This example illustrates binary classification. + * For information on multiclass classification, please refer to the JavaDecisionTree.java + * example. + */ + private static void testClassification(JavaRDD trainingData, + JavaRDD testData) { + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + Integer numClasses = 2; + HashMap categoricalFeaturesInfo = new HashMap(); + Integer numTrees = 3; // Use more in practice. + String featureSubsetStrategy = "auto"; // Let the algorithm choose. + String impurity = "gini"; + Integer maxDepth = 4; + Integer maxBins = 32; + Integer seed = 12345; + + final RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double testErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override + public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / testData.count(); + System.out.println("Test Error: " + testErr); + System.out.println("Learned classification forest model:\n" + model.toDebugString()); + } + + private static void testRegression(JavaRDD trainingData, + JavaRDD testData) { + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + HashMap categoricalFeaturesInfo = new HashMap(); + Integer numTrees = 3; // Use more in practice. + String featureSubsetStrategy = "auto"; // Let the algorithm choose. + String impurity = "variance"; + Integer maxDepth = 4; + Integer maxBins = 32; + Integer seed = 12345; + + final RandomForestModel model = RandomForest.trainRegressor(trainingData, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double testMSE = + predictionAndLabel.map(new Function, Double>() { + @Override + public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override + public Double call(Double a, Double b) { + return a + b; + } + }) / testData.count(); + System.out.println("Test Mean Squared Error: " + testMSE); + System.out.println("Learned regression forest model:\n" + model.toDebugString()); + } + + public static void main(String[] args) { + SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestExample"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + System.out.println("\nRunning example of classification using RandomForest\n"); + testClassification(trainingData, testData); + + System.out.println("\nRunning example of regression using RandomForest\n"); + testRegression(trainingData, testData); + sc.stop(); + } +} diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index 61ea4e06ecf3a..fccabd841b139 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -106,8 +106,7 @@ def reindexClassLabels(data): def usage(): print >> sys.stderr, \ - "Usage: decision_tree_runner [libsvm format data filepath]\n" + \ - " Note: This only supports binary classification." + "Usage: decision_tree_runner [libsvm format data filepath]" exit(1) @@ -127,16 +126,20 @@ def usage(): # Re-index class labels if needed. (reindexedData, origToNewLabels) = reindexClassLabels(points) + numClasses = len(origToNewLabels) # Train a classifier. categoricalFeaturesInfo = {} # no categorical features - model = DecisionTree.trainClassifier(reindexedData, numClasses=2, + model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. print "Trained DecisionTree for classification:" - print " Model numNodes: %d\n" % model.numNodes() - print " Model depth: %d\n" % model.depth() - print " Training accuracy: %g\n" % getAccuracy(model, reindexedData) - print model + print " Model numNodes: %d" % model.numNodes() + print " Model depth: %d" % model.depth() + print " Training accuracy: %g" % getAccuracy(model, reindexedData) + if model.numNodes() < 20: + print model.toDebugString() + else: + print model sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py new file mode 100755 index 0000000000000..d3c24f7664329 --- /dev/null +++ b/examples/src/main/python/mllib/random_forest_example.py @@ -0,0 +1,89 @@ +# +# 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. +# + +""" +Random Forest classification and regression using MLlib. + +Note: This example illustrates binary classification. + For information on multiclass classification, please refer to the decision_tree_runner.py + example. +""" + +import sys + +from pyspark.context import SparkContext +from pyspark.mllib.tree import RandomForest +from pyspark.mllib.util import MLUtils + + +def testClassification(trainingData, testData): + # Train a RandomForest model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + # Note: Use larger numTrees in practice. + # Setting featureSubsetStrategy="auto" lets the algorithm choose. + model = RandomForest.trainClassifier(trainingData, numClasses=2, + categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='gini', maxDepth=4, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count()\ + / float(testData.count()) + print('Test Error = ' + str(testErr)) + print('Learned classification forest model:') + print(model.toDebugString()) + + +def testRegression(trainingData, testData): + # Train a RandomForest model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + # Note: Use larger numTrees in practice. + # Setting featureSubsetStrategy="auto" lets the algorithm choose. + model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='variance', maxDepth=4, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum()\ + / float(testData.count()) + print('Test Mean Squared Error = ' + str(testMSE)) + print('Learned regression forest model:') + print(model.toDebugString()) + + +if __name__ == "__main__": + if len(sys.argv) > 1: + print >> sys.stderr, "Usage: random_forest_example" + exit(1) + sc = SparkContext(appName="PythonRandomForestExample") + + # Load and parse the data file into an RDD of LabeledPoint. + data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + print('\nRunning example of classification using RandomForest\n') + testClassification(trainingData, testData) + + print('\nRunning example of regression using RandomForest\n') + testRegression(trainingData, testData) + + sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala new file mode 100644 index 0000000000000..ce6bc066bd70d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -0,0 +1,110 @@ +/* + * 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.ml + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} +import org.apache.spark.sql.{Row, SQLContext} + +/** + * A simple example demonstrating model selection using CrossValidator. + * This example also demonstrates how Pipelines are Estimators. + * + * This example uses the [[LabeledDocument]] and [[Document]] case classes from + * [[SimpleTextClassificationPipeline]]. + * + * Run with + * {{{ + * bin/run-example ml.CrossValidatorExample + * }}} + */ +object CrossValidatorExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("CrossValidatorExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Prepare training documents, which are labeled. + val training = sparkContext.parallelize(Seq( + LabeledDocument(0L, "a b c d e spark", 1.0), + LabeledDocument(1L, "b d", 0.0), + LabeledDocument(2L, "spark f g h", 1.0), + LabeledDocument(3L, "hadoop mapreduce", 0.0), + LabeledDocument(4L, "b spark who", 1.0), + LabeledDocument(5L, "g d a y", 0.0), + LabeledDocument(6L, "spark fly", 1.0), + LabeledDocument(7L, "was mapreduce", 0.0), + LabeledDocument(8L, "e spark program", 1.0), + LabeledDocument(9L, "a e c l", 0.0), + LabeledDocument(10L, "spark compile", 1.0), + LabeledDocument(11L, "hadoop software", 0.0))) + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") + val hashingTF = new HashingTF() + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") + val lr = new LogisticRegression() + .setMaxIter(10) + val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + + // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. + // This will allow us to jointly choose parameters for all Pipeline stages. + // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + val crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator) + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, + // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. + val paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) + .addGrid(lr.regParam, Array(0.1, 0.01)) + .build() + crossval.setEstimatorParamMaps(paramGrid) + crossval.setNumFolds(2) // Use 3+ in practice + + // Run cross-validation, and choose the best set of parameters. + val cvModel = crossval.fit(training) + + // Prepare test documents, which are unlabeled. + val test = sparkContext.parallelize(Seq( + Document(4L, "spark i j k"), + Document(5L, "l m n"), + Document(6L, "mapreduce spark"), + Document(7L, "apache hadoop"))) + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + cvModel.transform(test) + .select('id, 'text, 'score, 'prediction) + .collect() + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala new file mode 100644 index 0000000000000..44d5b084c269a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.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.examples.ml + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.sql.{Row, SQLContext} + +/** + * A simple example demonstrating ways to specify parameters for Estimators and Transformers. + * Run with + * {{{ + * bin/run-example ml.SimpleParamsExample + * }}} + */ +object SimpleParamsExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("SimpleParamsExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Prepare training data. + // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans + // into SchemaRDDs, where it uses the bean metadata to infer the schema. + val training = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) + + // Create a LogisticRegression instance. This instance is an Estimator. + val lr = new LogisticRegression() + // Print out the parameters, documentation, and any default values. + println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") + + // We may set parameters using setter methods. + lr.setMaxIter(10) + .setRegParam(0.01) + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + val model1 = lr.fit(training) + // Since model1 is a Model (i.e., a Transformer produced by an Estimator), + // we can view the parameters it used during fit(). + // This prints the parameter (name: value) pairs, where names are unique IDs for this + // LogisticRegression instance. + println("Model 1 was fit using parameters: " + model1.fittingParamMap) + + // We may alternatively specify parameters using a ParamMap, + // which supports several methods for specifying parameters. + val paramMap = ParamMap(lr.maxIter -> 20) + paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. + paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. + + // One can also combine ParamMaps. + val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Change output column name + val paramMapCombined = paramMap ++ paramMap2 + + // Now learn a new model using the paramMapCombined parameters. + // paramMapCombined overrides all parameters set earlier via lr.set* methods. + val model2 = lr.fit(training, paramMapCombined) + println("Model 2 was fit using parameters: " + model2.fittingParamMap) + + // Prepare test documents. + val test = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) + + // Make predictions on test documents using the Transformer.transform() method. + // LogisticRegression.transform will only use the 'features' column. + // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' + // column since we renamed the lr.scoreCol parameter previously. + model2.transform(test) + .select('features, 'label, 'probability, 'prediction) + .collect() + .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => + println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index ee7897d9062d9..92895a05e479a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -20,10 +20,11 @@ package org.apache.spark.examples.ml import scala.beans.BeanInfo import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{Row, SQLContext} @BeanInfo case class LabeledDocument(id: Long, text: String, label: Double) @@ -81,6 +82,8 @@ object SimpleTextClassificationPipeline { model.transform(test) .select('id, 'text, 'score, 'prediction) .collect() - .foreach(println) + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + } } } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 54953adb5f3df..205d80dd02682 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -278,7 +278,7 @@ object DecisionTreeRunner { impurity = impurityCalculator, maxDepth = params.maxDepth, maxBins = params.maxBins, - numClassesForClassification = numClasses, + numClasses = numClasses, minInstancesPerNode = params.minInstancesPerNode, minInfoGain = params.minInfoGain, useNodeIdCache = params.useNodeIdCache, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index 1def8b45a230c..431ead8c0c165 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -103,7 +103,7 @@ object GradientBoostedTreesRunner { params.dataFormat, params.testInput, Algo.withName(params.algo), params.fracTest) val boostingStrategy = BoostingStrategy.defaultParams(params.algo) - boostingStrategy.treeStrategy.numClassesForClassification = numClasses + boostingStrategy.treeStrategy.numClasses = numClasses boostingStrategy.numIterations = params.numIterations boostingStrategy.treeStrategy.maxDepth = params.maxDepth diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index e545df1e37b9c..081a574beea5d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -162,11 +162,15 @@ class PipelineModel private[ml] ( } override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { - transformSchema(dataset.schema, paramMap, logging = true) - stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, paramMap)) + // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap + val map = (fittingParamMap ++ this.paramMap) ++ paramMap + transformSchema(dataset.schema, map, logging = true) + stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) } private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, paramMap)) + // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap + val map = (fittingParamMap ++ this.paramMap) ++ paramMap + stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, map)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 490e6609ad311..23fbd228d01cb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -18,16 +18,14 @@ package org.apache.spark.ml import scala.annotation.varargs -import scala.reflect.runtime.universe.TypeTag import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.api.java.JavaSchemaRDD -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.Star -import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.catalyst.expressions.ScalaUdf import org.apache.spark.sql.catalyst.types._ /** @@ -86,7 +84,7 @@ abstract class Transformer extends PipelineStage with Params { * Abstract class for transformers that take one input column, apply transformation, and output the * result as a new column. */ -private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransformer[IN, OUT, T]] +private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] extends Transformer with HasInputCol with HasOutputCol with Logging { def setInputCol(value: String): T = set(inputCol, value).asInstanceOf[T] @@ -99,6 +97,11 @@ private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransfor */ protected def createTransformFunc(paramMap: ParamMap): IN => OUT + /** + * Returns the data type of the output column. + */ + protected def outputDataType: DataType + /** * Validates the input type. Throw an exception if it is invalid. */ @@ -111,9 +114,8 @@ private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransfor if (schema.fieldNames.contains(map(outputCol))) { throw new IllegalArgumentException(s"Output column ${map(outputCol)} already exists.") } - val output = ScalaReflection.schemaFor[OUT] val outputFields = schema.fields :+ - StructField(map(outputCol), output.dataType, output.nullable) + StructField(map(outputCol), outputDataType, !outputDataType.isPrimitive) StructType(outputFields) } @@ -121,7 +123,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransfor transformSchema(dataset.schema, paramMap, logging = true) import dataset.sqlContext._ val map = this.paramMap ++ paramMap - val udf = this.createTransformFunc(map) - dataset.select(Star(None), udf.call(map(inputCol).attr) as map(outputCol)) + val udf = ScalaUdf(this.createTransformFunc(map), outputDataType, Seq(map(inputCol).attr)) + dataset.select(Star(None), udf as map(outputCol)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index b98b1755a3584..e0bfb1e484a2e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -21,7 +21,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{VectorUDT, Vector} +import org.apache.spark.sql.catalyst.types.DataType /** * :: AlphaComponent :: @@ -39,4 +40,6 @@ class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) hashingTF.transform } + + override protected def outputDataType: DataType = new VectorUDT() } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 0a6599b64c011..9352f40f372d3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param.ParamMap -import org.apache.spark.sql.{DataType, StringType} +import org.apache.spark.sql.{DataType, StringType, ArrayType} /** * :: AlphaComponent :: @@ -36,4 +36,6 @@ class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { protected override def validateInputType(inputType: DataType): Unit = { require(inputType == StringType, s"Input type must be string type but got $inputType.") } + + override protected def outputDataType: DataType = new ArrayType(StringType, false) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 8fd46aef4b99d..04f9cfb1bfc2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -17,13 +17,12 @@ package org.apache.spark.ml.param -import java.lang.reflect.Modifier - -import org.apache.spark.annotation.AlphaComponent - import scala.annotation.varargs import scala.collection.mutable +import java.lang.reflect.Modifier + +import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Identifiable /** @@ -222,6 +221,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Puts a list of param pairs (overwrites if the input params exists). */ + @varargs def put(paramPairs: ParamPair[_]*): this.type = { paramPairs.foreach { p => put(p.param.asInstanceOf[Param[Any]], p.value) @@ -282,6 +282,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * where the latter overwrites this if there exists conflicts. */ def ++(other: ParamMap): ParamMap = { + // TODO: Provide a better method name for Java users. new ParamMap(this.map ++ other.map) } @@ -290,6 +291,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * Adds all parameters from the input param map into this param map. */ def ++=(other: ParamMap): this.type = { + // TODO: Provide a better method name for Java users. this.map ++= other.map this } 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 481084f413da6..34b50ddbad28d 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 @@ -481,7 +481,7 @@ class PythonMLLibAPI extends Serializable { algo = algo, impurity = impurity, maxDepth = maxDepth, - numClassesForClassification = numClasses, + numClasses = numClasses, maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap, minInstancesPerNode = minInstancesPerNode, @@ -517,7 +517,7 @@ class PythonMLLibAPI extends Serializable { algo = algo, impurity = impurity, maxDepth = maxDepth, - numClassesForClassification = numClasses, + numClasses = numClasses, maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap) val cached = data.rdd.persist(StorageLevel.MEMORY_AND_DISK) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 8c4c9c6cf6ae2..9fed513becddc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -96,7 +96,9 @@ private[spark] object BLAS extends Serializable with Logging { * dot(x, y) */ def dot(x: Vector, y: Vector): Double = { - require(x.size == y.size) + require(x.size == y.size, + "BLAS.dot(x: Vector, y:Vector) was given Vectors with non-matching sizes:" + + " x.size = " + x.size + ", y.size = " + y.size) (x, y) match { case (dx: DenseVector, dy: DenseVector) => dot(dx, dy) 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 3d91867c896d9..73e7e32c6db31 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 @@ -136,7 +136,7 @@ object DecisionTree extends Serializable with Logging { * @param impurity impurity criterion used for information gain calculation * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification number of classes for classification. Default value of 2. + * @param numClasses number of classes for classification. Default value of 2. * @return DecisionTreeModel that can be used for prediction */ def train( @@ -144,8 +144,8 @@ object DecisionTree extends Serializable with Logging { algo: Algo, impurity: Impurity, maxDepth: Int, - numClassesForClassification: Int): DecisionTreeModel = { - val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification) + numClasses: Int): DecisionTreeModel = { + val strategy = new Strategy(algo, impurity, maxDepth, numClasses) new DecisionTree(strategy).run(input) } @@ -164,7 +164,7 @@ object DecisionTree extends Serializable with Logging { * @param impurity criterion used for information gain calculation * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification number of classes for classification. Default value of 2. + * @param numClasses number of classes for classification. Default value of 2. * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles * @param categoricalFeaturesInfo Map storing arity of categorical features. @@ -177,11 +177,11 @@ object DecisionTree extends Serializable with Logging { algo: Algo, impurity: Impurity, maxDepth: Int, - numClassesForClassification: Int, + numClasses: Int, maxBins: Int, quantileCalculationStrategy: QuantileStrategy, categoricalFeaturesInfo: Map[Int,Int]): DecisionTreeModel = { - val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification, maxBins, + val strategy = new Strategy(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, categoricalFeaturesInfo) new DecisionTree(strategy).run(input) } @@ -191,7 +191,7 @@ object DecisionTree extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels should take values {0, 1, ..., numClasses-1}. - * @param numClassesForClassification number of classes for classification. + * @param numClasses number of classes for classification. * @param categoricalFeaturesInfo Map storing arity of categorical features. * E.g., an entry (n -> k) indicates that feature n is categorical * with k categories indexed from 0: {0, 1, ..., k-1}. @@ -206,13 +206,13 @@ object DecisionTree extends Serializable with Logging { */ def trainClassifier( input: RDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: Map[Int, Int], impurity: String, maxDepth: Int, maxBins: Int): DecisionTreeModel = { val impurityType = Impurities.fromString(impurity) - train(input, Classification, impurityType, maxDepth, numClassesForClassification, maxBins, Sort, + train(input, Classification, impurityType, maxDepth, numClasses, maxBins, Sort, categoricalFeaturesInfo) } @@ -221,12 +221,12 @@ object DecisionTree extends Serializable with Logging { */ def trainClassifier( input: JavaRDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], impurity: String, maxDepth: Int, maxBins: Int): DecisionTreeModel = { - trainClassifier(input.rdd, numClassesForClassification, + trainClassifier(input.rdd, numClasses, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, impurity, maxDepth, maxBins) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 482d3395516e7..e9304b5e5c650 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -56,7 +56,7 @@ import org.apache.spark.util.Utils * etc. * @param numTrees If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt" for classification and @@ -269,7 +269,7 @@ object RandomForest extends Serializable with Logging { * @param strategy Parameters for training each tree in the forest. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt". @@ -293,13 +293,13 @@ object RandomForest extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels should take values {0, 1, ..., numClasses-1}. - * @param numClassesForClassification number of classes for classification. + * @param numClasses number of classes for classification. * @param categoricalFeaturesInfo Map storing arity of categorical features. * E.g., an entry (n -> k) indicates that feature n is categorical * with k categories indexed from 0: {0, 1, ..., k-1}. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt". @@ -315,7 +315,7 @@ object RandomForest extends Serializable with Logging { */ def trainClassifier( input: RDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: Map[Int, Int], numTrees: Int, featureSubsetStrategy: String, @@ -325,7 +325,7 @@ object RandomForest extends Serializable with Logging { seed: Int = Utils.random.nextInt()): RandomForestModel = { val impurityType = Impurities.fromString(impurity) val strategy = new Strategy(Classification, impurityType, maxDepth, - numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo) + numClasses, maxBins, Sort, categoricalFeaturesInfo) trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed) } @@ -334,7 +334,7 @@ object RandomForest extends Serializable with Logging { */ def trainClassifier( input: JavaRDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], numTrees: Int, featureSubsetStrategy: String, @@ -342,7 +342,7 @@ object RandomForest extends Serializable with Logging { maxDepth: Int, maxBins: Int, seed: Int): RandomForestModel = { - trainClassifier(input.rdd, numClassesForClassification, + trainClassifier(input.rdd, numClasses, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) } @@ -355,7 +355,7 @@ object RandomForest extends Serializable with Logging { * @param strategy Parameters for training each tree in the forest. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "onethird". @@ -384,7 +384,7 @@ object RandomForest extends Serializable with Logging { * with k categories indexed from 0: {0, 1, ..., k-1}. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "onethird". diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index e703adbdbfbb3..cf51d041c65a9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -51,7 +51,7 @@ case class BoostingStrategy( private[tree] def assertValid(): Unit = { treeStrategy.algo match { case Classification => - require(treeStrategy.numClassesForClassification == 2, + require(treeStrategy.numClasses == 2, "Only binary classification is supported for boosting.") case Regression => // nothing @@ -80,12 +80,12 @@ object BoostingStrategy { treeStrategy.maxDepth = 3 algo match { case "Classification" => - treeStrategy.numClassesForClassification = 2 + treeStrategy.numClasses = 2 new BoostingStrategy(treeStrategy, LogLoss) case "Regression" => new BoostingStrategy(treeStrategy, SquaredError) case _ => - throw new IllegalArgumentException(s"$algo is not supported by the boosting.") + throw new IllegalArgumentException(s"$algo is not supported by boosting.") } } } 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 d75f38433c081..d5cd89ab94e81 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 @@ -37,7 +37,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]]. * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification Number of classes for classification. + * @param numClasses Number of classes for classification. * (Ignored for regression.) * Default value is 2 (binary classification). * @param maxBins Maximum number of bins used for discretizing continuous features and @@ -73,7 +73,7 @@ class Strategy ( @BeanProperty var algo: Algo, @BeanProperty var impurity: Impurity, @BeanProperty var maxDepth: Int, - @BeanProperty var numClassesForClassification: Int = 2, + @BeanProperty var numClasses: Int = 2, @BeanProperty var maxBins: Int = 32, @BeanProperty var quantileCalculationStrategy: QuantileStrategy = Sort, @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), @@ -86,7 +86,7 @@ class Strategy ( @BeanProperty var checkpointInterval: Int = 10) extends Serializable { def isMulticlassClassification = - algo == Classification && numClassesForClassification > 2 + algo == Classification && numClasses > 2 def isMulticlassWithCategoricalFeatures = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) @@ -97,10 +97,10 @@ class Strategy ( algo: Algo, impurity: Impurity, maxDepth: Int, - numClassesForClassification: Int, + numClasses: Int, maxBins: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]) { - this(algo, impurity, maxDepth, numClassesForClassification, maxBins, Sort, + this(algo, impurity, maxDepth, numClasses, maxBins, Sort, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) } @@ -117,8 +117,8 @@ class Strategy ( */ def setCategoricalFeaturesInfo( categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]): Unit = { - setCategoricalFeaturesInfo( - categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) + this.categoricalFeaturesInfo = + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap } /** @@ -128,9 +128,9 @@ class Strategy ( private[tree] def assertValid(): Unit = { algo match { case Classification => - require(numClassesForClassification >= 2, - s"DecisionTree Strategy for Classification must have numClassesForClassification >= 2," + - s" but numClassesForClassification = $numClassesForClassification.") + require(numClasses >= 2, + s"DecisionTree Strategy for Classification must have numClasses >= 2," + + s" but numClasses = $numClasses.") require(Set(Gini, Entropy).contains(impurity), s"DecisionTree Strategy given invalid impurity for Classification: $impurity." + s" Valid settings: Gini, Entropy") @@ -160,7 +160,7 @@ class Strategy ( /** Returns a shallow copy of this instance. */ def copy: Strategy = { - new Strategy(algo, impurity, maxDepth, numClassesForClassification, maxBins, + new Strategy(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, categoricalFeaturesInfo, minInstancesPerNode, minInfoGain, maxMemoryInMB, subsamplingRate, useNodeIdCache, checkpointDir, checkpointInterval) } @@ -176,9 +176,9 @@ object Strategy { def defaultStrategy(algo: String): Strategy = algo match { case "Classification" => new Strategy(algo = Classification, impurity = Gini, maxDepth = 10, - numClassesForClassification = 2) + numClasses = 2) case "Regression" => new Strategy(algo = Regression, impurity = Variance, maxDepth = 10, - numClassesForClassification = 0) + numClasses = 0) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index 5bc0f2635c6b1..951733fada6be 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -110,7 +110,7 @@ private[tree] object DecisionTreeMetadata extends Logging { val numFeatures = input.take(1)(0).features.size val numExamples = input.count() val numClasses = strategy.algo match { - case Classification => strategy.numClassesForClassification + case Classification => strategy.numClasses case Regression => 0 } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 972c905ec9ffa..9347eaf9221a8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -57,7 +57,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 2, + numClasses = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) @@ -81,7 +81,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 2, + numClasses = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) @@ -177,7 +177,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 100, + numClasses = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) @@ -271,7 +271,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 100, + numClasses = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) // 2^(10-1) - 1 > 100, so categorical features will be ordered @@ -295,7 +295,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val strategy = new Strategy( Classification, Gini, - numClassesForClassification = 2, + numClasses = 2, maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) @@ -377,7 +377,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -401,7 +401,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -426,7 +426,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -451,7 +451,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -485,7 +485,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) val rootNode1 = modelOneNode.topNode.deepCopy() val rootNode2 = modelOneNode.topNode.deepCopy() @@ -545,7 +545,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) @@ -568,7 +568,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 2) + numClasses = 2) val model = DecisionTree.train(rdd, strategy) DecisionTreeSuite.validateClassifier(model, arr, 1.0) @@ -585,7 +585,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 2) + numClasses = 2) val model = DecisionTree.train(rdd, strategy) DecisionTreeSuite.validateClassifier(model, arr, 1.0) @@ -600,7 +600,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = maxBins, + numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -629,7 +629,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 100) + numClasses = 3, maxBins = 100) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -650,7 +650,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) + numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(metadata.isUnordered(featureIndex = 0)) @@ -671,7 +671,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 100, + numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -692,7 +692,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 10, + numClasses = 3, maxBins = 10, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) @@ -708,7 +708,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, - maxDepth = 2, numClassesForClassification = 2, minInstancesPerNode = 2) + maxDepth = 2, numClasses = 2, minInstancesPerNode = 2) val model = DecisionTree.train(rdd, strategy) assert(model.topNode.isLeaf) @@ -737,7 +737,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), - numClassesForClassification = 2, minInstancesPerNode = 2) + numClasses = 2, minInstancesPerNode = 2) val rootNode = DecisionTree.train(rdd, strategy).topNode @@ -755,7 +755,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, minInfoGain = 1.0) + numClasses = 2, minInfoGain = 1.0) val model = DecisionTree.train(input, strategy) assert(model.topNode.isLeaf) @@ -781,7 +781,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, - numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 -> 3)) + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) @@ -824,7 +824,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 -> 3)) + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index d4d54cf4c9e2a..3aa97e544680b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -100,7 +100,7 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) val treeStrategy = new Strategy(algo = Classification, impurity = Variance, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = Map.empty, + numClasses = 2, categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) val boostingStrategy = new BoostingStrategy(treeStrategy, LogLoss, numIterations, learningRate) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 90a8c2dfdab80..f7f0f20c6c125 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -57,7 +57,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) binaryClassificationTestWithContinuousFeatures(strategy) } @@ -65,7 +65,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) binaryClassificationTestWithContinuousFeatures(strategy) } @@ -93,7 +93,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Regression, impurity = Variance, - maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + maxDepth = 2, maxBins = 10, numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) regressionTestWithContinuousFeatures(strategy) } @@ -102,7 +102,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Regression, impurity = Variance, - maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + maxDepth = 2, maxBins = 10, numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) regressionTestWithContinuousFeatures(strategy) } @@ -169,14 +169,14 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { test("Binary classification with continuous features: subsampling features") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) } test("Binary classification with continuous features and node Id cache: subsampling features") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) } @@ -191,7 +191,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) + numClasses = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, featureSubsetStrategy = "sqrt", seed = 12345) EnsembleTestHelper.validateClassifier(model, arr, 1.0) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b16ed66aeb3c3..6ff08723772aa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -365,7 +365,10 @@ object Unidoc { "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", - "mllib.tree.impurity", "mllib.tree.model", "mllib.util" + "mllib.tree.impurity", "mllib.tree.model", "mllib.util", + "mllib.evaluation", "mllib.feature", "mllib.random", "mllib.stat.correlation", + "mllib.stat.test", "mllib.tree.impl", "mllib.tree.loss", + "ml", "ml.classification", "ml.evaluation", "ml.feature", "ml.param", "ml.tuning" ), "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), "-noqualifier", "java.lang" diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 46e253991aa56..66702478474dc 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -250,7 +250,7 @@ def _train(cls, data, algo, numClasses, categoricalFeaturesInfo, numTrees, return RandomForestModel(model) @classmethod - def trainClassifier(cls, data, numClassesForClassification, categoricalFeaturesInfo, numTrees, + def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, featureSubsetStrategy="auto", impurity="gini", maxDepth=4, maxBins=32, seed=None): """ @@ -259,7 +259,7 @@ def trainClassifier(cls, data, numClassesForClassification, categoricalFeaturesI :param data: Training dataset: RDD of LabeledPoint. Labels should take values {0, 1, ..., numClasses-1}. - :param numClassesForClassification: number of classes for classification. + :param numClasses: number of classes for classification. :param categoricalFeaturesInfo: Map storing arity of categorical features. E.g., an entry (n -> k) indicates that feature n is categorical with k categories indexed from 0: {0, 1, ..., k-1}. @@ -320,7 +320,7 @@ def trainClassifier(cls, data, numClassesForClassification, categoricalFeaturesI >>> model.predict(rdd).collect() [1.0, 0.0] """ - return cls._train(data, "classification", numClassesForClassification, + return cls._train(data, "classification", numClasses, categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) diff --git a/sbt/sbt b/sbt/sbt index c172fa74bc771..0a251d97db95c 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -71,8 +71,8 @@ Usage: $script_name [options] -Dkey=val pass -Dkey=val directly to the java runtime -J-X pass option -X directly to the java runtime (-J is stripped) - -S-X add -X to sbt's scalacOptions (-J is stripped) - -PmavenProfiles Enable a maven profile for the build. + -S-X add -X to sbt's scalacOptions (-S is stripped) + -PmavenProfiles Enable a maven profile for the build. In the case of duplicated or conflicting options, the order above shows precedence: JAVA_OPTS lowest, command line options highest. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 84a3567895175..08b982bc671e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -45,9 +45,9 @@ case class Coalesce(children: Seq[Expression]) extends Expression { override def eval(input: Row): Any = { var i = 0 var result: Any = null - while(i < children.size && result == null) { - result = children(i).eval(input) - i += 1 + val childIterator = children.iterator + while (childIterator.hasNext && result == null) { + result = childIterator.next().eval(input) } result } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 9b89c3bfb3307..14f8659f15b3f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -191,7 +191,10 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) val selectedPartitions = partitions.filter(p => partitionFilters.forall(_(p))) val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) val selectedFiles = selectedPartitions.flatMap(_.files).map(f => fs.makeQualified(f.getPath)) - org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles:_*) + // FileInputFormat cannot handle empty lists. + if (selectedFiles.nonEmpty) { + org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles: _*) + } // Push down filters when possible predicates diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index f83e647014193..bcebce7603f13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -22,7 +22,6 @@ import java.util.TimeZone import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /* Implicits */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 56fc85239e1c0..edf291f917f07 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} +import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan, PhysicalRDD} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation @@ -104,53 +104,61 @@ private[hive] trait HiveStrategies { case a: AttributeReference => UnresolvedAttribute(a.name) }) - if (relation.hiveQlTable.isPartitioned) { - val rawPredicate = pruningPredicates.reduceOption(And).getOrElse(Literal(true)) - // Translate the predicate so that it automatically casts the input values to the correct - // data types during evaluation - val castedPredicate = rawPredicate transform { - case a: AttributeReference => - val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId) - val key = relation.partitionKeys(idx) - Cast(BoundReference(idx, StringType, nullable = true), key.dataType) - } - - val inputData = new GenericMutableRow(relation.partitionKeys.size) - val pruningCondition = - if(codegenEnabled) { - GeneratePredicate(castedPredicate) - } else { - InterpretedPredicate(castedPredicate) + try { + if (relation.hiveQlTable.isPartitioned) { + val rawPredicate = pruningPredicates.reduceOption(And).getOrElse(Literal(true)) + // Translate the predicate so that it automatically casts the input values to the + // correct data types during evaluation. + val castedPredicate = rawPredicate transform { + case a: AttributeReference => + val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId) + val key = relation.partitionKeys(idx) + Cast(BoundReference(idx, StringType, nullable = true), key.dataType) } - val partitions = relation.hiveQlPartitions.filter { part => - val partitionValues = part.getValues - var i = 0 - while (i < partitionValues.size()) { - inputData(i) = partitionValues(i) - i += 1 + val inputData = new GenericMutableRow(relation.partitionKeys.size) + val pruningCondition = + if (codegenEnabled) { + GeneratePredicate(castedPredicate) + } else { + InterpretedPredicate(castedPredicate) + } + + val partitions = relation.hiveQlPartitions.filter { part => + val partitionValues = part.getValues + var i = 0 + while (i < partitionValues.size()) { + inputData(i) = partitionValues(i) + i += 1 + } + pruningCondition(inputData) } - pruningCondition(inputData) - } - hiveContext - .parquetFile(partitions.map(_.getLocation).mkString(",")) - .addPartitioningAttributes(relation.partitionKeys) - .lowerCase - .where(unresolvedOtherPredicates) - .select(unresolvedProjection:_*) - .queryExecution - .executedPlan - .fakeOutput(projectList.map(_.toAttribute)):: Nil - } else { - hiveContext - .parquetFile(relation.hiveQlTable.getDataLocation.toString) - .lowerCase - .where(unresolvedOtherPredicates) - .select(unresolvedProjection:_*) - .queryExecution - .executedPlan - .fakeOutput(projectList.map(_.toAttribute)) :: Nil + hiveContext + .parquetFile(partitions.map(_.getLocation).mkString(",")) + .addPartitioningAttributes(relation.partitionKeys) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection: _*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } else { + hiveContext + .parquetFile(relation.hiveQlTable.getDataLocation.toString) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection: _*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } + } catch { + // parquetFile will throw an exception when there is no data. + // TODO: Remove this hack for Spark 1.3. + case iae: java.lang.IllegalArgumentException + if iae.getMessage.contains("Can not create a Path from an empty string") => + PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil } case _ => Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 7159ebd0353ad..488ebba043794 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -218,6 +218,12 @@ abstract class ParquetTest extends QueryTest with BeforeAndAfterAll { 10) } + test(s"non-existant partition $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE p = 1000"), + 0) + } + test(s"multi-partition pruned count $table") { checkAnswer( sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"), diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index f95d72379171c..290d9943a5077 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} +import java.io.{File, FilenameFilter} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, ListBuffer, Map} @@ -223,10 +224,51 @@ private[spark] trait ClientBase extends Logging { } } } + if (cachedSecondaryJarLinks.nonEmpty) { sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) } + /** + * Do the same for datanucleus jars, if they exist in spark home. Find all datanucleus-* jars, + * copy them to the remote fs, and add them to the class path. + * + * This is necessary because the datanucleus jars cannot be included in the assembly jar due + * to metadata conflicts involving plugin.xml. At the time of writing, these are the only + * jars that cannot be distributed with the uber jar and have to be treated differently. + * + * For more details, see SPARK-2624, and https://github.com/apache/spark/pull/3238 + */ + for (libsDir <- dataNucleusJarsDir(sparkConf)) { + val libsURI = new URI(libsDir) + val jarLinks = ListBuffer.empty[String] + if (libsURI.getScheme != LOCAL_SCHEME) { + val localPath = getQualifiedLocalPath(libsURI) + val localFs = FileSystem.get(localPath.toUri, hadoopConf) + if (localFs.exists(localPath)) { + val jars = localFs.listFiles(localPath, /* recursive */ false) + while (jars.hasNext) { + val jar = jars.next() + val name = jar.getPath.getName + if (name.startsWith("datanucleus-")) { + // copy to remote and add to classpath + val src = jar.getPath + val destPath = copyFileToRemote(dst, src, replication) + distCacheMgr.addResource(localFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, name, statCache) + jarLinks += name + } + } + } + } else { + jarLinks += libsURI.toString + Path.SEPARATOR + "*" + } + + if (jarLinks.nonEmpty) { + sparkConf.set(CONF_SPARK_DATANUCLEUS_JARS, jarLinks.mkString(",")) + } + } + localResources } @@ -551,6 +593,13 @@ private[spark] object ClientBase extends Logging { // Internal config to propagate the location of the user's jar to the driver/executors val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" + // Location of the datanucleus jars + val CONF_SPARK_DATANUCLEUS_DIR = "spark.yarn.datanucleus.dir" + + // Internal config to propagate the locations of datanucleus jars found to add to the + // classpath of the executors. Value should be a comma-separated list of paths to each jar. + val CONF_SPARK_DATANUCLEUS_JARS = "spark.yarn.datanucleus.jars" + // Internal config to propagate the locations of any extra jars to add to the classpath // of the executors val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" @@ -583,6 +632,19 @@ private[spark] object ClientBase extends Logging { } } + /** + * Find the user-defined provided jars directory if configured, or return SPARK_HOME/lib if not. + * + * This method first looks for $CONF_SPARK_DATANUCLEUS_DIR inside the SparkConf, then looks for + * Spark home inside the the SparkConf and the user environment. + */ + private def dataNucleusJarsDir(conf: SparkConf): Option[String] = { + conf.getOption(CONF_SPARK_DATANUCLEUS_DIR).orElse { + val sparkHome = conf.getOption("spark.home").orElse(sys.env.get("SPARK_HOME")) + sparkHome.map(path => path + Path.SEPARATOR + "lib") + } + } + /** * Return the path to the given application's staging directory. */ @@ -684,6 +746,13 @@ private[spark] object ClientBase extends Logging { addUserClasspath(args, sparkConf, env) } + // Add datanucleus jars to classpath + for (entries <- sparkConf.getOption(CONF_SPARK_DATANUCLEUS_JARS)) { + entries.split(",").filter(_.nonEmpty).foreach { entry => + addFileToClasspath(entry, null, env) + } + } + // Append all jar files under the working directory to the classpath. addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 17b79ae1d82c4..b055e9b72dc61 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.net.URI import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -104,6 +105,81 @@ class ClientBaseSuite extends FunSuite with Matchers { cp should not contain (ClientBase.APP_JAR) } + test("DataNucleus in classpath") { + val dnJars = "local:/dn/core.jar,/dn/api.jar" + val conf = new Configuration() + val sparkConf = new SparkConf() + .set(ClientBase.CONF_SPARK_JAR, SPARK) + .set(ClientBase.CONF_SPARK_DATANUCLEUS_JARS, dnJars) + val env = new MutableHashMap[String, String]() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + ClientBase.populateClasspath(args, conf, sparkConf, env) + + val cp = env("CLASSPATH").split(File.pathSeparator) + s"$dnJars".split(",").foreach({ entry => + val uri = new URI(entry) + if (ClientBase.LOCAL_SCHEME.equals(uri.getScheme())) { + cp should contain (uri.getPath()) + } else { + cp should not contain (uri.getPath()) + } + }) + } + + test("DataNucleus using local:") { + val dnDir = "local:/datanucleus" + val conf = new Configuration() + val sparkConf = new SparkConf() + .set(ClientBase.CONF_SPARK_JAR, SPARK) + .set(ClientBase.CONF_SPARK_DATANUCLEUS_DIR, dnDir) + val yarnConf = new YarnConfiguration() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean()) + + val tempDir = Utils.createTempDir() + try { + client.prepareLocalResources(tempDir.getAbsolutePath()) + val jars = sparkConf.get(ClientBase.CONF_SPARK_DATANUCLEUS_JARS).split(",") + val uri = new URI(dnDir) + jars should contain (uri.toString + Path.SEPARATOR + "*") + } finally { + Utils.deleteRecursively(tempDir) + } + } + + test("DataNucleus using file:") { + val dnDir = Utils.createTempDir() + val tempDir = Utils.createTempDir() + + try { + // create mock datanucleus jar + val tempJar = File.createTempFile("datanucleus-", null, dnDir) + + val conf = new Configuration() + val sparkConf = new SparkConf() + .set(ClientBase.CONF_SPARK_JAR, SPARK) + .set(ClientBase.CONF_SPARK_DATANUCLEUS_DIR, dnDir.toURI.toString) + val yarnConf = new YarnConfiguration() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean()) + + client.prepareLocalResources(tempDir.getAbsolutePath()) + + val jars = sparkConf.get(ClientBase.CONF_SPARK_DATANUCLEUS_JARS).split(",") + jars should contain (tempJar.getName) + } finally { + Utils.deleteRecursively(dnDir) + Utils.deleteRecursively(tempDir) + } + } + test("Jar path propagation through SparkConf") { val conf = new Configuration() val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK)