From 4f4bf8a714e90b606a99addfa5ef1916e9e766db Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 7 Nov 2014 13:34:59 -0800 Subject: [PATCH 01/34] Wrote rowToColumnStoreDense method --- .../org/apache/spark/mllib/tree/AltDT.scala | 180 ++++++++++++++++++ 1 file changed, 180 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala new file mode 100644 index 000000000000..b2723e814a45 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala @@ -0,0 +1,180 @@ +package org.apache.spark.mllib.tree + +import scala.collection.mutable + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{DenseVector, Vectors, Vector} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} +import org.apache.spark.rdd.RDD + + +/** + * DecisionTree which partitions data by feature. + * + * Algorithm: + * - Repartition data, grouping by feature. + * - Prep data (sort continuous features). + * - On each partition, initialize instance--node map with each instance at root node. + * - Iterate, training 1 new level of the tree at a time: + * - On each partition, + * - For each node, select best split (among that partition's features). + * - Aggregate best split for each node. + * - Aggregate bit vector (1 bit/instance) indicating whether each instance splits + * left or right. + * - Broadcast bit vector. On each partition, update instance--node map. + */ +@Experimental +class AltDT (private val strategy: Strategy) extends Serializable with Logging { + + strategy.assertValid() + + /** + * Convert a dataset of [[DenseVector]] from row storage to column storage. + * + * WARNING: This shuffles the ENTIRE dataset across the network, so it is a VERY EXPENSIVE + * operation. This can also fail if 1 column is too large to fit on 1 partition. + * + * This maintains sparsity in the data. + * + * This maintains matrix structure. I.e., each partition of the output RDD holds adjacent + * columns. The number of partitions will be min(input RDD's number of partitions, numColumns). + * + * @param rowStore The input vectors are data rows/instances. + * @return RDD of (columnIndex, columnValues) pairs, + * where each pair corresponds to one entire column. + * If either dimension of the given data is 0, this returns an empty RDD. + * If vector lengths do not match, this throws an exception. + * + * TODO: Add implementation for sparse data. + * For sparse data, distribute more evenly based on number of non-zeros. + * (First collect stats to decide how to partition.) + * TODO: Move elsewhere in MLlib. + */ + def rowToColumnStoreDense(rowStore: RDD[Vector]): RDD[(Int, Vector)] = { + /* + val numRows = { + val longNumRows: Long = rowStore.count() + require(longNumRows < Int.MaxValue, s"rowToColumnStore given RDD with $longNumRows rows," + + s" but can handle at most ${Int.MaxValue} rows") + longNumRows.toInt + } + */ + val numCols = rowStore.take(1)(0).size + val numSourcePartitions = rowStore.partitions.size + val numTargetPartitions = Math.min(numCols, rowStore.partitions.size) + if (numTargetPartitions == 0) { + return rowStore.sparkContext.parallelize(Seq.empty[(Int, DenseVector)]) + } + val maxColumnsPerPartition = Math.floor(numCols / (numTargetPartitions + 0.0)).toInt + + def getNumColsInGroup(groupIndex: Int) = { + if (groupIndex + 1 < numTargetPartitions) { + maxColumnsPerPartition + } else { + numCols - (numTargetPartitions - 1) * maxColumnsPerPartition // last partition + } + } + + /* On each partition, re-organize into groups of columns: + (groupIndex, (sourcePartitionIndex, partCols)), + where partCols(colIdx) = partial column. + The groupIndex will be used to groupByKey. + The sourcePartitionIndex is used to ensure instance indices match up after the shuffle. + The partial columns will be stacked into full columns after the shuffle. + Note: By design, partCols will always have at least 1 column. + */ + val partialColumns: RDD[(Int, (Int, Array[Array[Double]]))] = + rowStore.mapPartitionsWithIndex { case (sourcePartitionIndex, iterator) => + // columnSets(groupIndex)(colIdx) + // = column values for each instance in sourcePartitionIndex, + // where colIdx is a 0-based index for columns for groupIndex + val columnSets = new Array[Array[mutable.ArrayBuffer[Double]]](numTargetPartitions) + Range(0, numTargetPartitions).foreach { groupIndex => + columnSets(groupIndex) = + new Array[mutable.ArrayBuffer[Double]](getNumColsInGroup(groupIndex)) + } + iterator.foreach { row => + Range(0, numTargetPartitions).foreach { groupIndex => + val fromCol = groupIndex * maxColumnsPerPartition + val numColsInTargetPartition = getNumColsInGroup(groupIndex) + var colIdx = 0 + while (colIdx < numColsInTargetPartition) { + columnSets(groupIndex)(colIdx) += row(fromCol + colIdx) + colIdx += 1 + } + } + } + Range(0, numTargetPartitions).map { groupIndex => + (groupIndex, + (sourcePartitionIndex, columnSets(groupIndex).map(_.toArray))) + }.toIterator + } + + // Shuffle data + val groupedPartialColumns: RDD[(Int, Iterable[(Int, Array[Array[Double]])])] = + partialColumns.groupByKey() + + // Each target partition now holds its set of columns. + // Group the partial columns into full columns. + val fullColumns = groupedPartialColumns.flatMap { case (groupIndex, iterator) => + // We do not know the number of rows per group, so we need to collect the groups + // before filling the full columns. + val collectedPartCols = new Array[Array[Array[Double]]](numSourcePartitions) + iterator.foreach { case (sourcePartitionIndex, partCols) => + collectedPartCols(sourcePartitionIndex) = partCols + } + val rowOffsets: Array[Int] = collectedPartCols.map(_(0).size).scanLeft(0)(_ + _) + val numRows = rowOffsets.last + // Initialize full columns + val fromCol = groupIndex * maxColumnsPerPartition + val numColumnsInPartition = getNumColsInGroup(groupIndex) + val partitionColumns: Array[Array[Double]] = + Array.fill[Array[Double]](numColumnsInPartition)(new Array[Double](numRows)) + var colIdx = 0 // index within group + while (colIdx < numColumnsInPartition) { + var sourcePartitionIndex = 0 + while (sourcePartitionIndex < numSourcePartitions) { + val partColLength = + rowOffsets(sourcePartitionIndex + 1) - rowOffsets(sourcePartitionIndex) + Array.copy(collectedPartCols(sourcePartitionIndex)(colIdx), 0, + partitionColumns(colIdx), rowOffsets(sourcePartitionIndex), partColLength) + sourcePartitionIndex += 1 + } + colIdx += 1 + } + val columnIndices = Range(0, numColumnsInPartition).map(_ + fromCol) + val columns = partitionColumns.map(Vectors.dense) + columnIndices.zip(columns) + } + + fullColumns + } + + /** + * This uses [[rowToColumnStore()]] for shuffling the features, so it comes with the same + * guarantees. This then groups each partition's features together into an array and pairs it + * with a vector of all labels. + * @param rowStore Dataset in row-storage format + * @return RDD of + */ + def rowToColumnStoreWithLabels( + rowStore: RDD[LabeledPoint]): RDD[(Vector, Array[(Int, Vector)])] = { + } + + /** + * Method to train a decision tree model over an RDD + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @return DecisionTreeModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint]): DecisionTreeModel = { + + } + +} + +object AltDT extends Serializable with Logging { +} From b8d2c93061fa09286dd7a071c00db8b0894e4e2d Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 7 Nov 2014 15:28:41 -0800 Subject: [PATCH 02/34] Wrote TreeUtilSuite to test rowToColumnStoreDense --- .../tree/{AltDT.scala => impl/Util.scala} | 75 +++++----------- .../spark/mllib/tree/impl/TreeUtilSuite.scala | 88 +++++++++++++++++++ 2 files changed, 111 insertions(+), 52 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/tree/{AltDT.scala => impl/Util.scala} (74%) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala similarity index 74% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala rename to mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala index b2723e814a45..eea91e036ca8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala @@ -1,36 +1,30 @@ -package org.apache.spark.mllib.tree +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{DenseVector, Vectors, Vector} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.rdd.RDD -/** - * DecisionTree which partitions data by feature. - * - * Algorithm: - * - Repartition data, grouping by feature. - * - Prep data (sort continuous features). - * - On each partition, initialize instance--node map with each instance at root node. - * - Iterate, training 1 new level of the tree at a time: - * - On each partition, - * - For each node, select best split (among that partition's features). - * - Aggregate best split for each node. - * - Aggregate bit vector (1 bit/instance) indicating whether each instance splits - * left or right. - * - Broadcast bit vector. On each partition, update instance--node map. - */ -@Experimental -class AltDT (private val strategy: Strategy) extends Serializable with Logging { - - strategy.assertValid() +private[tree] object Util { /** * Convert a dataset of [[DenseVector]] from row storage to column storage. @@ -55,14 +49,15 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { * TODO: Move elsewhere in MLlib. */ def rowToColumnStoreDense(rowStore: RDD[Vector]): RDD[(Int, Vector)] = { - /* val numRows = { val longNumRows: Long = rowStore.count() require(longNumRows < Int.MaxValue, s"rowToColumnStore given RDD with $longNumRows rows," + s" but can handle at most ${Int.MaxValue} rows") longNumRows.toInt } - */ + if (numRows == 0) { + return rowStore.sparkContext.parallelize(Seq.empty[(Int, DenseVector)]) + } val numCols = rowStore.take(1)(0).size val numSourcePartitions = rowStore.partitions.size val numTargetPartitions = Math.min(numCols, rowStore.partitions.size) @@ -95,7 +90,7 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { val columnSets = new Array[Array[mutable.ArrayBuffer[Double]]](numTargetPartitions) Range(0, numTargetPartitions).foreach { groupIndex => columnSets(groupIndex) = - new Array[mutable.ArrayBuffer[Double]](getNumColsInGroup(groupIndex)) + Array.fill[mutable.ArrayBuffer[Double]](getNumColsInGroup(groupIndex))(mutable.ArrayBuffer[Double]()) } iterator.foreach { row => Range(0, numTargetPartitions).foreach { groupIndex => @@ -153,28 +148,4 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { fullColumns } - - /** - * This uses [[rowToColumnStore()]] for shuffling the features, so it comes with the same - * guarantees. This then groups each partition's features together into an array and pairs it - * with a vector of all labels. - * @param rowStore Dataset in row-storage format - * @return RDD of - */ - def rowToColumnStoreWithLabels( - rowStore: RDD[LabeledPoint]): RDD[(Vector, Array[(Int, Vector)])] = { - } - - /** - * Method to train a decision tree model over an RDD - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @return DecisionTreeModel that can be used for prediction - */ - def train(input: RDD[LabeledPoint]): DecisionTreeModel = { - - } - -} - -object AltDT extends Serializable with Logging { } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala new file mode 100644 index 000000000000..652bfabfe63a --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.tree.impl.Util._ +import org.apache.spark.mllib.util.LocalSparkContext +import org.scalatest.FunSuite + +/** + * Test suite for [[org.apache.spark.mllib.tree.impl.Util]]. + */ +class TreeUtilSuite extends FunSuite with LocalSparkContext { + + private def check(rows: Seq[Vector]): Unit = { + val numRowPartitions = 2 + val rowStore = sc.parallelize(rows, numRowPartitions) + val colStore = rowToColumnStoreDense(rowStore) + val numColPartitions = colStore.partitions.size + val cols: Map[Int, Vector] = colStore.collect().toMap + val numRows = rows.size + if (numRows == 0) { + assert(cols.size == 0) + return + } + val numCols = rows(0).size + if (numCols == 0) { + assert(cols.size == 0) + return + } + rows.zipWithIndex.foreach { case (row, i) => + var j = 0 + while (j < numCols) { + assert(row(j) == cols(j)(i)) + j += 1 + } + } + val expectedNumColPartitions = Math.min(rowStore.partitions.size, numCols) + assert(numColPartitions === expectedNumColPartitions) + } + + test("rowToColumnStoreDense: small") { + val rows = Seq( + Vectors.dense(1.0, 2.0, 3.0, 4.0), + Vectors.dense(1.1, 2.1, 3.1, 4.1), + Vectors.dense(1.2, 2.2, 3.2, 4.2) + ) + check(rows) + } + + test("rowToColumnStoreDense: large") { + var numRows = 100 + var numCols = 90 + val rows = Range(0, numRows).map { i => + Vectors.dense(Range(0, numCols).map(_ + numCols * i + 0.0).toArray) + } + check(rows) + } + + test("rowToColumnStoreDense: 0 rows") { + val rows = Seq.empty[Vector] + check(rows) + } + + test("rowToColumnStoreDense: 0 cols") { + val rows = Seq( + Vectors.dense(Array.empty[Double]), + Vectors.dense(Array.empty[Double]), + Vectors.dense(Array.empty[Double]) + ) + check(rows) + } +} From 100d5064e32eebcd0810501310fa23a6d4f994f9 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 7 Nov 2014 15:33:57 -0800 Subject: [PATCH 03/34] scala style --- .../scala/org/apache/spark/mllib/tree/impl/Util.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala index eea91e036ca8..a6095aacec51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.tree.impl -import scala.collection.mutable - import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{DenseVector, Vectors, Vector} import org.apache.spark.rdd.RDD @@ -49,6 +47,8 @@ private[tree] object Util { * TODO: Move elsewhere in MLlib. */ def rowToColumnStoreDense(rowStore: RDD[Vector]): RDD[(Int, Vector)] = { + import scala.collection.mutable.ArrayBuffer + val numRows = { val longNumRows: Long = rowStore.count() require(longNumRows < Int.MaxValue, s"rowToColumnStore given RDD with $longNumRows rows," + @@ -87,10 +87,10 @@ private[tree] object Util { // columnSets(groupIndex)(colIdx) // = column values for each instance in sourcePartitionIndex, // where colIdx is a 0-based index for columns for groupIndex - val columnSets = new Array[Array[mutable.ArrayBuffer[Double]]](numTargetPartitions) + val columnSets = new Array[Array[ArrayBuffer[Double]]](numTargetPartitions) Range(0, numTargetPartitions).foreach { groupIndex => columnSets(groupIndex) = - Array.fill[mutable.ArrayBuffer[Double]](getNumColsInGroup(groupIndex))(mutable.ArrayBuffer[Double]()) + Array.fill[ArrayBuffer[Double]](getNumColsInGroup(groupIndex))(ArrayBuffer[Double]()) } iterator.foreach { row => Range(0, numTargetPartitions).foreach { groupIndex => From 74d6732698e55aa1ba4dbe01c35ec9d040a51abb Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 30 Nov 2014 18:22:15 -0500 Subject: [PATCH 04/34] Added rowToColumnStoreSparse to tree Utils, and added test suite for it. --- .../apache/spark/mllib/tree/impl/Util.scala | 244 +++++++++++++++++- .../spark/mllib/tree/impl/TreeUtilSuite.scala | 113 ++++++-- 2 files changed, 336 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala index a6095aacec51..f30f3fb0c2e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala @@ -17,15 +17,18 @@ package org.apache.spark.mllib.tree.impl +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.{DenseVector, Vectors, Vector} +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vectors, Vector} import org.apache.spark.rdd.RDD private[tree] object Util { /** - * Convert a dataset of [[DenseVector]] from row storage to column storage. + * Convert a dataset of [[Vector]] from row storage to column storage. + * This can take any [[Vector]] type but stores data as [[DenseVector]]. * * WARNING: This shuffles the ENTIRE dataset across the network, so it is a VERY EXPENSIVE * operation. This can also fail if 1 column is too large to fit on 1 partition. @@ -47,7 +50,6 @@ private[tree] object Util { * TODO: Move elsewhere in MLlib. */ def rowToColumnStoreDense(rowStore: RDD[Vector]): RDD[(Int, Vector)] = { - import scala.collection.mutable.ArrayBuffer val numRows = { val longNumRows: Long = rowStore.count() @@ -56,13 +58,13 @@ private[tree] object Util { longNumRows.toInt } if (numRows == 0) { - return rowStore.sparkContext.parallelize(Seq.empty[(Int, DenseVector)]) + return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) } val numCols = rowStore.take(1)(0).size val numSourcePartitions = rowStore.partitions.size - val numTargetPartitions = Math.min(numCols, rowStore.partitions.size) + val numTargetPartitions = Math.min(numCols, numSourcePartitions) if (numTargetPartitions == 0) { - return rowStore.sparkContext.parallelize(Seq.empty[(Int, DenseVector)]) + return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) } val maxColumnsPerPartition = Math.floor(numCols / (numTargetPartitions + 0.0)).toInt @@ -96,6 +98,7 @@ private[tree] object Util { Range(0, numTargetPartitions).foreach { groupIndex => val fromCol = groupIndex * maxColumnsPerPartition val numColsInTargetPartition = getNumColsInGroup(groupIndex) + // TODO: match-case here on row as Dense or Sparse Vector (for speed) var colIdx = 0 while (colIdx < numColsInTargetPartition) { columnSets(groupIndex)(colIdx) += row(fromCol + colIdx) @@ -148,4 +151,233 @@ private[tree] object Util { fullColumns } + + /** + * This checks for an empty RDD (0 rows or 0 columns). + * This will throw an exception if any columns have non-matching numbers of features. + * @param rowStore Dataset of vectors which all have the same length (number of columns). + * @return Array over columns of the number of non-zero elements in each column. + * Returns empty array if the RDD is empty. + */ + def countNonZerosPerColumn(rowStore: RDD[Vector]): Array[Long] = { + val firstRow = rowStore.take(1) + if (firstRow.size == 0) { + return Array.empty[Long] + } + val numCols = firstRow(0).size + val colSizes: Array[Long] = rowStore.mapPartitions { iterator => + val partColSizes = Array.fill[Long](numCols)(0) + iterator.foreach { + case dv: DenseVector => + var col = 0 + while (col < dv.size) { + if (dv(col) != 0.0) partColSizes(col) += 1 + col += 1 + } + case sv: SparseVector => + var k = 0 + while (k < sv.indices.size) { + if (sv.values(k) != 0.0) partColSizes(sv.indices(k)) += 1 + k += 1 + } + } + Iterator(partColSizes) + }.fold(Array.fill[Long](numCols)(0)){ + case (v1, v2) => v1.zip(v2).map(v12 => v12._1 + v12._2) + } + colSizes + } + + /** + * The returned RDD sets the number of partitions as follows: + * - The targeted number is: + * numTargetPartitions = min(rowStore num partitions, num columns) * overPartitionFactor. + * - The actual number will be in the range [numTargetPartitions, 2 * numTargetPartitions]. + * Partitioning is done such that each partition holds consecutive columns. + * + * @param rowStore RDD of dataset rows + * @param overPartitionFactor Multiplier for the targeted number of partitions. This parameter + * helps to ensure that P partitions handled by P compute cores + * do not get split into slightly more than P partitions; + * if that occurred, then work would not be shared evenly. + * @return RDD of (column index, column) pairs + */ + def rowToColumnStoreSparse( + rowStore: RDD[Vector], + overPartitionFactor: Int = 3): RDD[(Int, Vector)] = { + + val numRows = { + val longNumRows: Long = rowStore.count() + require(longNumRows < Int.MaxValue, s"rowToColumnStore given RDD with $longNumRows rows," + + s" but can handle at most ${Int.MaxValue} rows") + longNumRows.toInt + } + if (numRows == 0) { + return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) + } + + // Compute the number of non-zeros in each column. + val colSizes: Array[Long] = countNonZerosPerColumn(rowStore) + val numCols = colSizes.size + val numSourcePartitions = rowStore.partitions.size + if (numCols == 0 || numSourcePartitions == 0) { + return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) + } + val totalNonZeros = colSizes.sum + + // Split columns into groups. + // Groups are chosen greedily and sequentially, putting as many columns as possible in each + // group (limited by the number of non-zeros). Try to limit the number of non-zeros per + // group to at most targetNonZerosPerPartition. + val numTargetPartitions = math.min(numSourcePartitions, numCols) * overPartitionFactor + val targetNonZerosPerPartition = (totalNonZeros / numTargetPartitions.toDouble).floor.toLong + val groupStartColumns: Array[Int] = { + val startCols = new ArrayBuffer[Int]() + startCols += 0 + var currentStartCol = 0 + var currentNonZeros: Long = 0 + var col = 0 + while (col < numCols) { + if (currentNonZeros >= targetNonZerosPerPartition && col != startCols.last) { + startCols += col + currentStartCol = col + currentNonZeros = 0 + } else { + currentNonZeros += colSizes(col) + } + col += 1 + } + startCols += numCols + startCols.toArray + } + val numGroups = groupStartColumns.size - 1 // actual number of destination partitions + + /* On each partition, re-organize into groups of columns: + (groupIndex, (sourcePartitionIndex, partCols)), + where partCols(colIdx) = partial column. + The groupIndex will be used to groupByKey. + The sourcePartitionIndex is used to ensure instance indices match up after the shuffle. + The partial columns will be stacked into full columns after the shuffle. + Note: By design, partCols will always have at least 1 column. + */ + val partialColumns: RDD[(Int, (Int, Array[SparseVector]))] = + rowStore.zipWithIndex().mapPartitionsWithIndex { case (sourcePartitionIndex, iterator) => + type SparseVectorBuffer = (Int, ArrayBuffer[Int], ArrayBuffer[Double]) + // columnSets(groupIndex)(colIdx) + // = column values for each instance in sourcePartitionIndex, + // where colIdx is a 0-based index for columns for groupIndex, + // and where column values are in sparse format: (size, indices, values) + val columnSetSizes = new Array[Array[Int]](numGroups) + val columnSetIndices = new Array[Array[ArrayBuffer[Int]]](numGroups) + val columnSetValues = new Array[Array[ArrayBuffer[Double]]](numGroups) + Range(0, numGroups).foreach { groupIndex => + val numColsInGroup = groupStartColumns(groupIndex + 1) - groupStartColumns(groupIndex) + columnSetSizes(groupIndex) = Array.fill[Int](numColsInGroup)(0) + columnSetIndices(groupIndex) = + Array.fill[ArrayBuffer[Int]](numColsInGroup)(new ArrayBuffer[Int]) + columnSetValues(groupIndex) = + Array.fill[ArrayBuffer[Double]](numColsInGroup)(new ArrayBuffer[Double]) + } + iterator.foreach { + case (dv: DenseVector, rowIndex: Long) => + Range(0, numGroups).foreach { groupIndex => + val fromCol = groupStartColumns(groupIndex) + val numColsInGroup = groupStartColumns(groupIndex + 1) - groupStartColumns(groupIndex) + var colIdx = 0 + while (colIdx < numColsInGroup) { + columnSetSizes(groupIndex)(colIdx) += 1 + columnSetIndices(groupIndex)(colIdx) += rowIndex.toInt + columnSetValues(groupIndex)(colIdx) += dv(fromCol + colIdx) + colIdx += 1 + } + } + case (sv: SparseVector, rowIndex: Long) => + /* + A sparse vector is chopped into groups (destination partitions). + We iterate through the non-zeros (indexed by k), going to the next group sv.indices(k) + passes the current group's boundary. + */ + var groupIndex = 0 + var k = 0 // index into SparseVector non-zeros + val nnz = sv.indices.size + while (groupIndex < numGroups && k < nnz) { + val fromColumn = groupStartColumns(groupIndex) + val groupEndColumn = groupStartColumns(groupIndex + 1) + while (k < nnz && sv.indices(k) < groupEndColumn) { + val columnIndex = sv.indices(k) // index in full row + val colIdx = columnIndex - fromColumn // index in group of columns + columnSetSizes(groupIndex)(colIdx) += 1 + columnSetIndices(groupIndex)(colIdx) += rowIndex.toInt + columnSetValues(groupIndex)(colIdx) += sv.values(k) + k += 1 + } + groupIndex += 1 + } + } + Range(0, numGroups).map { groupIndex => + val numColsInGroup = groupStartColumns(groupIndex + 1) - groupStartColumns(groupIndex) + val groupPartialColumns: Array[SparseVector] = Range(0, numColsInGroup).map { colIdx => + new SparseVector(columnSetSizes(groupIndex)(colIdx), + columnSetIndices(groupIndex)(colIdx).toArray, + columnSetValues(groupIndex)(colIdx).toArray) + }.toArray + (groupIndex, (sourcePartitionIndex, groupPartialColumns)) + }.toIterator + } + + // Shuffle data + val groupedPartialColumns: RDD[(Int, Iterable[(Int, Array[SparseVector])])] = + partialColumns.groupByKey() + + // Each target partition now holds its set of columns. + // Group the partial columns into full columns. + val fullColumns = groupedPartialColumns.flatMap { case (groupIndex, iterator) => + val numColsInGroup = groupStartColumns(groupIndex + 1) - groupStartColumns(groupIndex) + + // We do not know the number of rows or non-zeros per group, so we need to collect the groups + // before filling the full columns. + // collectedPartCols(sourcePartitionIndex)(colIdx) = partial column + val collectedPartCols = new Array[Array[SparseVector]](numSourcePartitions) + // nzCounts(colIdx)(sourcePartitionIndex) = number of non-zeros + val nzCounts = Array.fill[Array[Int]](numColsInGroup)(Array.fill[Int](numSourcePartitions)(0)) + iterator.foreach { case (sourcePartitionIndex, partCols) => + collectedPartCols(sourcePartitionIndex) = partCols + partCols.zipWithIndex.foreach { case (partCol, colIdx) => + nzCounts(colIdx)(sourcePartitionIndex) += partCol.indices.size + } + } + // nzOffsets(colIdx)(sourcePartitionIndex) = cumulative number of non-zeros + val nzOffsets: Array[Array[Int]] = nzCounts.map(_.scanLeft(0)(_ + _)) + + // Initialize full columns + val columnNZIndices: Array[Array[Int]] = + nzOffsets.map(colNZOffsets => new Array[Int](colNZOffsets.last)) + val columnNZValues: Array[Array[Double]] = + nzOffsets.map(colNZOffsets => new Array[Double](colNZOffsets.last)) + + // Fill columns + var colIdx = 0 // index within group + while (colIdx < numColsInGroup) { + var sourcePartitionIndex = 0 + while (sourcePartitionIndex < numSourcePartitions) { + val nzStartOffset = nzOffsets(colIdx)(sourcePartitionIndex) + val partColLength = nzOffsets(colIdx)(sourcePartitionIndex + 1) - nzStartOffset + Array.copy(collectedPartCols(sourcePartitionIndex)(colIdx).indices, 0, + columnNZIndices(colIdx), nzStartOffset, partColLength) + Array.copy(collectedPartCols(sourcePartitionIndex)(colIdx).values, 0, + columnNZValues(colIdx), nzStartOffset, partColLength) + sourcePartitionIndex += 1 + } + colIdx += 1 + } + val columns = columnNZIndices.zip(columnNZValues).map { case (indices, values) => + Vectors.sparse(numRows, indices, values) + } + val fromColumn = groupStartColumns(groupIndex) + val columnIndices = Range(0, numColsInGroup).map(_ + fromColumn) + columnIndices.zip(columns) + } + + fullColumns + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala index 652bfabfe63a..e22aec360b55 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala @@ -17,17 +17,20 @@ package org.apache.spark.mllib.tree.impl -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.tree.impl.Util._ -import org.apache.spark.mllib.util.LocalSparkContext +import scala.collection.mutable + import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors} +import org.apache.spark.mllib.tree.impl.Util._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + /** * Test suite for [[org.apache.spark.mllib.tree.impl.Util]]. */ -class TreeUtilSuite extends FunSuite with LocalSparkContext { +class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { - private def check(rows: Seq[Vector]): Unit = { + private def checkDense(rows: Seq[Vector]): Unit = { val numRowPartitions = 2 val rowStore = sc.parallelize(rows, numRowPartitions) val colStore = rowToColumnStoreDense(rowStore) @@ -50,39 +53,119 @@ class TreeUtilSuite extends FunSuite with LocalSparkContext { j += 1 } } - val expectedNumColPartitions = Math.min(rowStore.partitions.size, numCols) + val expectedNumColPartitions = math.min(rowStore.partitions.size, numCols) assert(numColPartitions === expectedNumColPartitions) } - test("rowToColumnStoreDense: small") { + private def checkSparse(rows: Seq[Vector]): Unit = { + val numRowPartitions = 2 + val overPartitionFactor = 2 + val rowStore = sc.parallelize(rows, numRowPartitions) + val colStore = rowToColumnStoreSparse(rowStore, overPartitionFactor) + val numColPartitions = colStore.partitions.size + val cols: Map[Int, Vector] = colStore.collect().toMap + val numRows = rows.size + // Check cases with 0 rows or cols + if (numRows == 0) { + assert(cols.size == 0) + return + } + val numCols = rows(0).size + if (numCols == 0) { + assert(cols.size == 0) + return + } + // Check values (and count non-zeros too) + var expectedNumNonZeros = 0 + rows.zipWithIndex.foreach { case (row, i) => + var j = 0 + while (j < numCols) { + assert(row(j) == cols(j)(i)) + if (row(j) != 0) expectedNumNonZeros += 1 + j += 1 + } + } + // Check sparsity + val numNonZeros = cols.values.map { + case sv: SparseVector => sv.indices.size + case _ => throw new RuntimeException( + "checkSparse() found column which was not converted to SparseVector.") + }.sum + assert(numNonZeros === expectedNumNonZeros) + // Check partitions to make sure they each contain consecutive columns. + val colsByPartition: Array[(Int, Array[(Int, Vector)])] = colStore.mapPartitionsWithIndex { + case (partitionIndex, iterator) => + val partCols = new mutable.ArrayBuffer[(Int, Vector)] + iterator.foreach(col => partCols += col) + Iterator((partitionIndex, iterator.toArray)) + }.collect() + colsByPartition.foreach { case (partitionIndex, partCols) => + var j = 0 + while (j + 1 < partCols.size) { + val curColIndex = partCols(j)._1 + val nextColIndex = partCols(j + 1)._1 + assert(curColIndex + 1 == nextColIndex) + j += 1 + } + } + } + + test("rowToColumnStore: small dense") { val rows = Seq( Vectors.dense(1.0, 2.0, 3.0, 4.0), Vectors.dense(1.1, 2.1, 3.1, 4.1), Vectors.dense(1.2, 2.2, 3.2, 4.2) ) - check(rows) + checkDense(rows) + checkSparse(rows) + } + + test("rowToColumnStore: small sparse") { + val rows = Seq( + Vectors.sparse(4, Array(0, 1), Array(1.0, 2.0)), + Vectors.sparse(4, Array(1, 2), Array(1.1, 2.1)), + Vectors.sparse(4, Array(2, 3), Array(1.2, 2.2)) + ) + checkDense(rows) + checkSparse(rows) } - test("rowToColumnStoreDense: large") { + test("rowToColumnStore: large dense") { + // Note: All values must be non-zero since rowToColumnStoreSparse() automatically ignores + // zero-valued elements. var numRows = 100 var numCols = 90 val rows = Range(0, numRows).map { i => - Vectors.dense(Range(0, numCols).map(_ + numCols * i + 0.0).toArray) + Vectors.dense(Range(0, numCols).map(_ + numCols * i + 1.0).toArray) } - check(rows) + checkDense(rows) + checkSparse(rows) + } + + test("rowToColumnStore: mixed dense and sparse") { + val rows = Seq( + Vectors.dense(1.0, 2.0, 3.0, 4.0), + Vectors.sparse(4, Array(1, 2), Array(1.1, 2.1)), + Vectors.dense(1.2, 2.2, 3.2, 4.2), + Vectors.sparse(4, Array(0, 2), Array(1.3, 2.3)) + ) + checkDense(rows) + checkSparse(rows) } - test("rowToColumnStoreDense: 0 rows") { + test("rowToColumnStore: 0 rows") { val rows = Seq.empty[Vector] - check(rows) + checkDense(rows) + checkSparse(rows) } - test("rowToColumnStoreDense: 0 cols") { + test("rowToColumnStore: 0 cols") { val rows = Seq( Vectors.dense(Array.empty[Double]), Vectors.dense(Array.empty[Double]), Vectors.dense(Array.empty[Double]) ) - check(rows) + checkDense(rows) + checkSparse(rows) } } From 1d1d71f425ffc233379c1e3bd381cd257ad78be4 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 22 Dec 2014 18:12:09 -0800 Subject: [PATCH 05/34] small fix in TreeUtilSuite --- .../org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala index e22aec360b55..de6f312f7d83 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala @@ -133,8 +133,8 @@ class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { test("rowToColumnStore: large dense") { // Note: All values must be non-zero since rowToColumnStoreSparse() automatically ignores // zero-valued elements. - var numRows = 100 - var numCols = 90 + val numRows = 100 + val numCols = 90 val rows = Range(0, numRows).map { i => Vectors.dense(Range(0, numCols).map(_ + numCols * i + 1.0).toArray) } From 3fcdeaf4df9d33ccdb2aef8389183d37736d7c14 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 23 Jul 2015 09:32:17 -0700 Subject: [PATCH 06/34] some updates before rebasing --- .../org/apache/spark/mllib/tree/AltDT.scala | 352 ++++++++++++++++++ .../apache/spark/mllib/tree/impl/Util.scala | 6 +- .../spark/mllib/tree/impurity/Entropy.scala | 21 ++ .../spark/mllib/tree/impurity/Gini.scala | 21 ++ .../spark/mllib/tree/impurity/Impurity.scala | 38 ++ .../spark/mllib/tree/impurity/Variance.scala | 17 + .../spark/mllib/tree/model/AltDTModel.scala | 26 ++ 7 files changed, 480 insertions(+), 1 deletion(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala new file mode 100644 index 000000000000..2c6fa09acbf0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala @@ -0,0 +1,352 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType +import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType +import org.apache.spark.mllib.tree.impurity._ +import org.apache.spark.storage.StorageLevel + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vectors, Vector} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} +import org.apache.spark.mllib.tree.impl.Util._ +import org.apache.spark.mllib.tree.model._ +import org.apache.spark.rdd.RDD + + +/** + * DecisionTree which partitions data by feature. + * + * Algorithm: + * - Repartition data, grouping by feature. + * - Prep data (sort continuous features). + * - On each partition, initialize instance--node map with each instance at root node. + * - Iterate, training 1 new level of the tree at a time: + * - On each partition, for each feature on the partition, select the best split for each node. + * - Aggregate best split for each node. + * - Aggregate bit vector (1 bit/instance) indicating whether each instance splits + * left or right. + * - Broadcast bit vector. On each partition, update instance--node map. + * + * TODO: Update to use a sparse column store. + */ +@Experimental +class AltDT (private val strategy: Strategy) extends Serializable with Logging { + + strategy.assertValid() + + /** + * Feature vector types are based on (feature type, representation). + * The feature type can be continuous or categorical. + * + * Features are sorted by value, so we must store indices + values. + * These values are currently stored in a dense representation only. + * TODO: Support sparse storage. + */ + private class FeatureVector( + val featureIndex: Int, + val featureType: FeatureType, + val values: Array[Double], + val indices: Array[Int]) + extends Serializable + + private object FeatureVector { + /** Store column sorted by feature values. */ + def fromOriginal( + featureIndex: Int, + featureType: FeatureType, + featureVector: Vector): FeatureVector = { + val (values, indices) = featureVector.toArray.zipWithIndex.sorted.unzip + new FeatureVector(featureIndex, featureType, values.toArray, indices.toArray) + } + } + + /** + * Intermediate data stored on each partition during learning. + * @param columns Subset of columns (features) stored in this partition. + * @param nodeOffsets Offsets into the rows (instances) indicating how they are split among + * nodes. The rows corresponding to node i are in the range + * [nodeOffsets(i), nodeOffsets(i+1)). + */ + private case class PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int]) + extends Serializable { + + /** + * Update columns and nodeOffsets for the next level of the tree.. + * @param bitVector Bit vector encoding splits for the next level of the tree. + * bitVector(i) = false iff instance i goes to the left child. + * @return Updated partition info + */ + def update(bitVector: Array[Boolean]): PartitionInfo = ??? + } + + private def getImpurityAggregator: ImpurityAggregatorSingle = { + strategy.impurity match { + case Entropy => new EntropyAggregatorSingle(strategy.numClasses) + case Gini => new GiniAggregatorSingle(strategy.numClasses) + case Variance => new VarianceAggregatorSingle + } + } + + /* + private class NodeIndex private (val splits: Array[Boolean]) extends Serializable { + import NodeIndex._ + + def leftChild: Array[Boolean] = splits :+ LEFT + + def rightChild: Array[Boolean] = splits :+ RIGHT + + } + + private object NodeIndex extends Serializable { + final val LEFT: Boolean = false + final val RIGHT: Boolean = true + def root: NodeIndex = new NodeIndex(Array.emptyBooleanArray) + } + + private class SplitInfo() extends Serializable + */ + + /** + * Method to train a decision tree model over an RDD + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @return DecisionTreeModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint]): DecisionTreeModel = { + // TODO: Check validity of params + + // The case with 1 node (depth = 0) is handled separately. + // This allows all iterations in the depth > 0 case to use the same code. + if (strategy.maxDepth == 0) { + val impurityAggregator: ImpurityAggregatorSingle = + input.aggregate(getImpurityAggregator)( + (agg, lp) => agg.update(lp.label, 1.0), + (agg1, agg2) => agg1.merge(agg2)) + val impurityCalculator = impurityAggregator.getCalculator + val node = Node(nodeIndex = 0, predict = impurityCalculator.getPredict, + impurity = impurityCalculator.calculate(), isLeaf = true) + return new DecisionTreeModel(node, strategy.algo) + } + + // Prepare column store. + // Note: rowToColumnStoreDense checks to make sure numRows < Int.MaxValue. + // TODO: Is this mapping from arrays to iterators to arrays (when constructing learningData)? + // Or is the mapping implicit (i.e., not costly)? + val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) + val numRows: Int = colStoreInit.take(1)(0)._2.size + val labels = new Array[Double](numRows) + input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => + labels(rowIndex.toInt) = label + } + val labelsBroadcast = input.sparkContext.broadcast(labels) + + // Sort each column by feature values. + val colStore: RDD[FeatureVector] = colStoreInit.map { case (featureIndex: Int, col: Vector) => + if (strategy.categoricalFeaturesInfo.contains(featureIndex)) { + FeatureVector.fromOriginal(featureIndex, FeatureType.Categorical, col) + } else { + FeatureVector.fromOriginal(featureIndex, FeatureType.Continuous, col) + } + } + // Group columns together into one array of columns per partition. + val groupedColStore: RDD[Array[FeatureVector]] = colStore.mapPartitions { iterator => + val groupedCols = new ArrayBuffer[FeatureVector] + iterator.foreach(groupedCols += _) + Iterator(groupedCols.toArray) + } + groupedColStore.persist(StorageLevel.MEMORY_AND_DISK) + + // Initialize partitions with 1 partition (i.e., each instance at the root node). + var partitionInfos: RDD[PartitionInfo] = groupedColStore.map { groupedCols => + new PartitionInfo(groupedCols, Array[Int](0, numRows)) + } + + // Iteratively learn, one level of the tree at a time. + var currentLevel = 0 + while (currentLevel < strategy.maxDepth) { + + val numNodesInLevel: Int = ??? + + // On each partition, for each feature on the partition, select the best split for each node. + // This will use: + // - groupedColStore (the features) + // - partitionInfos (the node -> instance mapping) + // - labelsBroadcast (the labels column) + // Each worker returns: + // for each node, best split + info gain + val workerBestSplitsAndGains: RDD[Array[(Split, InformationGainStats)]] = partitionInfos.map { + case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int]) => + val localLabels = labelsBroadcast.value + val localBestSplitsAndGains = new Array[(Split, InformationGainStats)](numNodesInLevel) + // Iterate over the nodes in the current level. + var nodeIndexInLevel = 0 + while (nodeIndexInLevel < numNodesInLevel) { + val fromOffset = nodeOffsets(nodeIndexInLevel) + val toOffset = nodeOffsets(nodeIndexInLevel + 1) + val splitsAndStats = + columns.map(col => chooseSplit(col, localLabels, fromOffset, toOffset)) + localBestSplitsAndGains(nodeIndexInLevel) = splitsAndStats.maxBy(_._2.gain) + nodeIndexInLevel += 1 + } + localBestSplitsAndGains + } + + // Aggregate best split for each node. + // TODO: verify that initialization gives info gain 0 + val bestSplitsAndGains: Array[(Split, InformationGainStats)] = + workerBestSplitsAndGains.fold(new Array[(Split, InformationGainStats)](numNodesInLevel)){ + case (splitsGains1, splitsGains2) => + splitsGains1.zip(splitsGains2).map { case ((split1, gain1), (split2, gain2)) => + if (gain1.gain >= gain2.gain) { + (split1, gain1) + } else { + (split2, gain2) + } + } + } + + // Update current model. + // TODO + + // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left or right. + // - Send chosen splits to workers. + // - Each worker creates part of the bit vector corresponding to the splits it created. + // - Aggregate the partial bit vectors to create one vector (of length numRows). + val broadcastBestSplits: Broadcast[Array[Split]] = + input.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) + val workerPartialBitVectors: RDD[Array[PartialBitVector]] = partitionInfos.map { + case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int]) => + val localBestSplits = broadcastBestSplits.value + // localFeatureIndex[feature index] = index into groupedCols + val localFeatureIndex: Map[Int, Int] = columns.map(_.featureIndex).zipWithIndex.toMap + localBestSplits.zipWithIndex.flatMap { case (split: Split, nodeIndexInLevel: Int) => + if (localFeatureIndex.contains(split.feature)) { + val fromOffset = nodeOffsets(nodeIndexInLevel) + val toOffset = nodeOffsets(nodeIndexInLevel + 1) + val colIndex: Int = localFeatureIndex(split.feature) + Iterator(PartialBitVector.fromSplit( + columns(colIndex), fromOffset, toOffset, split, nodeIndexInLevel)) + } else { + Iterator() + } + }.toArray + } + val fullBitVector: Array[Boolean] = { + val aggBitVector: Array[PartialBitVector] = + workerPartialBitVectors.fold(Array.empty[PartialBitVector])(PartialBitVector.merge) + assert(aggBitVector.size == 1) + assert(aggBitVector(0).fromNode == 0 && aggBitVector(0).toNode == numNodesInLevel) + aggBitVector(0).bits + } + + // Broadcast bit vector. On each partition, update instance--node map. + val broadcastBitVector = input.sparkContext.broadcast(fullBitVector) + partitionInfos = partitionInfos.map { + case partitionInfo: PartitionInfo => + val localBitVector: Array[Boolean] = broadcastBitVector.value + partitionInfo.update(localBitVector) + } + + currentLevel += 1 + } + + // Done with learning + groupedColStore.unpersist() + labelsBroadcast.unpersist() + // TODO: return model + } + + // TODO + // NOTE: The order of col does not correspond to the order of labels. Use the index in col. + private def chooseSplit( + col: FeatureVector, + labels: Array[Double], + fromOffset: Int, + toOffset: Int): (Split, InformationGainStats) = ??? + + private class PartialBitVector(val fromNode: Int, val toNode: Int, val bits: Array[Boolean]) + + private object PartialBitVector { + def merge( + parts1: Array[PartialBitVector], + parts2: Array[PartialBitVector]): Array[PartialBitVector] = { + // Merge sorted parts1, parts2 + val sortedParts = Array.fill[PartialBitVector](parts1.size + parts2.size)(null) + var i1 = 0 // indexing parts1 + var i2 = 0 // indexing parts2 + while (i1 < parts1.size || i2 < parts2.size) { + val i = i1 + i2 // index into sortedParts + if (i1 < parts1.size) { + if (i2 < parts2.size) { + // Choose between parts1,2 + if (parts1(i1).fromNode < parts2(i2).fromNode) { + sortedParts(i) = parts1(i1) + i1 += 1 + } else { + sortedParts(i) = parts2(i2) + i2 += 1 + } + } else { + // Take remaining items from parts1 + parts1.view.slice(i1, parts1.size).copyToArray(sortedParts, i) + i1 = parts1.size + } + } else { + // Take remaining items from parts2 + parts2.view.slice(i2, parts2.size).copyToArray(sortedParts, i) + i2 = parts2.size + } + } + // Merge adjacent PartialBitVectors (for adjacent node ranges) + val newParts = new ArrayBuffer[PartialBitVector]() + var j = 0 // indexing sortedParts + if (sortedParts.size > 0) { + newParts += sortedParts(0) + j += 1 + } + while (j < sortedParts.size) { + // Check to see if the next PartialBitVector can be merged with the previous one. + if (newParts.last.toNode == sortedParts(j).fromNode) { + // TODO: RIGHT HERE NOW + } else { + + } + j += 1 + } + newParts.toArray + } + + def fromSplit( + col: FeatureVector, + fromOffset: Int, + toOffset: Int, + split: Split, + nodeIndexInLevel: Int) = ??? + } + +} + +object AltDT extends Serializable with Logging { +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala index f30f3fb0c2e8..9fcc1a4fb208 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala @@ -159,7 +159,7 @@ private[tree] object Util { * @return Array over columns of the number of non-zero elements in each column. * Returns empty array if the RDD is empty. */ - def countNonZerosPerColumn(rowStore: RDD[Vector]): Array[Long] = { + private def countNonZerosPerColumn(rowStore: RDD[Vector]): Array[Long] = { val firstRow = rowStore.take(1) if (firstRow.size == 0) { return Array.empty[Long] @@ -195,6 +195,10 @@ private[tree] object Util { * - The actual number will be in the range [numTargetPartitions, 2 * numTargetPartitions]. * Partitioning is done such that each partition holds consecutive columns. * + * TODO: Update this to adaptively make columns dense or sparse based on a sparsity threshold. + * + * TODO: Cache rowStore temporarily. + * * @param rowStore RDD of dataset rows * @param overPartitionFactor Multiplier for the targeted number of partitions. This parameter * helps to ensure that P partitions handled by P compute cores diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 73df6b054a8c..4e06380049de 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -116,6 +116,27 @@ private[tree] class EntropyAggregator(numClasses: Int) } +/** + * Version of Entropy aggregator which owns its data and is only for one node. + */ +private[tree] class EntropyAggregatorSingle private (stats: Array[Double]) + extends ImpurityAggregatorSingle(stats) with Serializable { + + def this(numClasses: Int) = this(new Array[Double](numClasses)) + + def update(label: Double, instanceWeight: Double): this.type = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"EntropyAggregatorSingle given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + stats(label.toInt) += instanceWeight + this + } + + def getCalculator: EntropyCalculator = new EntropyCalculator(stats) + +} + /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[EntropyAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index f21845b21a80..f886bfc63515 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -112,6 +112,27 @@ private[tree] class GiniAggregator(numClasses: Int) } +/** + * Version of Gini aggregator which owns its data and is only for one node. + */ +private[tree] class GiniAggregatorSingle private (stats: Array[Double]) + extends ImpurityAggregatorSingle(stats) with Serializable { + + def this(numClasses: Int) = this(new Array[Double](numClasses)) + + def update(label: Double, instanceWeight: Double): this.type = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"GiniAggregatorSingle given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + stats(label.toInt) += instanceWeight + this + } + + def getCalculator: GiniCalculator = new GiniCalculator(stats) + +} + /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[GiniAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 4637dcceea7f..4f821406ad17 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.tree.impurity import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.mllib.tree.model.Predict /** * :: Experimental :: @@ -92,6 +93,37 @@ private[spark] abstract class ImpurityAggregator(val statsSize: Int) extends Ser } +/** + * Version of impurity aggregator which owns its data and is only for 1 node. + */ +private[tree] abstract class ImpurityAggregatorSingle(val stats: Array[Double]) + extends Serializable { + + def statsSize: Int = stats.size + + /** + * Merge two aggregators. + * @return This aggregator (modified). + */ + def merge(other: ImpurityAggregatorSingle): this.type = { + var i = 0 + while (i < statsSize) { + stats(i) += other.stats(i) + i += 1 + } + this + } + + /** + * Update stats with the given label. + * @return This aggregator (modified). + */ + def update(label: Double, instanceWeight: Double): this.type + + /** Get an [[ImpurityCalculator]] for the current stats. */ + def getCalculator: ImpurityCalculator +} + /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[ImpurityAggregator]], this class stores its own data and is for a specific @@ -158,6 +190,12 @@ private[spark] abstract class ImpurityCalculator(val stats: Array[Double]) exten */ def prob(label: Double): Double = -1 + /** Get [[Predict]] struct. */ + def getPredict = { + val pred = this.predict + new Predict(predict = pred, prob = this.prob(pred)) + } + /** * Return the index of the largest array element. * Fails if the array is empty. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index a74197278d6f..d6d7b65fb241 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -96,6 +96,23 @@ private[tree] class VarianceAggregator() } +/** + * Version of Variance aggregator which owns its data and is only for one node. + */ +private[tree] class VarianceAggregatorSingle + extends ImpurityAggregatorSingle(new Array[Double](3)) with Serializable { + + def update(label: Double, instanceWeight: Double): this.type = { + stats(0) += instanceWeight + stats(1) += instanceWeight * label + stats(2) += instanceWeight * label * label + this + } + + def getCalculator: VarianceCalculator = new VarianceCalculator(stats) + +} + /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[GiniAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala new file mode 100644 index 000000000000..1d9e0c7aedc8 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.model + +class AltDTModel { + +} + +class Prediction + +class AltDTNode(val prediction: Prediction, ) \ No newline at end of file From bfa181917d8a25fea1ad9005ed32b817122a2a51 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 23 Jul 2015 16:36:54 -0700 Subject: [PATCH 07/34] Done implement MVP for partitioning by feature, but need to test and debug --- .../tree/model => ml/tree}/AltDTModel.scala | 2 +- .../org/apache/spark/ml/tree/Split.scala | 18 + .../org/apache/spark/ml/tree/impl/AltDT.scala | 499 ++++++++++++++++++ .../spark/ml/tree/impl/BitSubvector.scala | 77 +++ .../spark/ml/tree/impl/InfoGainStats.scala | 70 +++ .../spark/ml/tree/impl/RandomForest.scala | 11 +- .../tree/impl/TreeUtil.scala} | 25 +- .../org/apache/spark/ml/tree/impurities.scala | 152 ++++++ .../org/apache/spark/mllib/tree/AltDT.scala | 352 ------------ .../mllib/tree/configuration/Strategy.scala | 2 +- .../spark/mllib/tree/impurity/Entropy.scala | 21 - .../spark/mllib/tree/impurity/Gini.scala | 21 - .../spark/mllib/tree/impurity/Impurity.scala | 31 -- .../spark/mllib/tree/impurity/Variance.scala | 17 - .../tree/impl/TreeUtilSuite.scala | 11 +- 15 files changed, 843 insertions(+), 466 deletions(-) rename mllib/src/main/scala/org/apache/spark/{mllib/tree/model => ml/tree}/AltDTModel.scala (95%) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/impl/InfoGainStats.scala rename mllib/src/main/scala/org/apache/spark/{mllib/tree/impl/Util.scala => ml/tree/impl/TreeUtil.scala} (96%) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala rename mllib/src/test/scala/org/apache/spark/{mllib => ml}/tree/impl/TreeUtilSuite.scala (94%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala similarity index 95% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala index 1d9e0c7aedc8..1cdd058178a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/AltDTModel.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.model +package org.apache.spark.ml.tree class AltDTModel { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index 78199cc2df58..34aa080a74a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -47,6 +47,12 @@ sealed trait Split extends Serializable { */ private[tree] def shouldGoLeft(binnedFeature: Int, splits: Array[Split]): Boolean + /** + * Return true (split to left) or false (split to right). + * @param feature Feature value (original value, not binned) + */ + private[tree] def shouldGoLeft(feature: Double): Boolean + /** Convert to old Split format */ private[tree] def toOld: OldSplit } @@ -112,6 +118,14 @@ final class CategoricalSplit private[ml] ( } } + override private[tree] def shouldGoLeft(feature: Double): Boolean = { + if (isLeft) { + categories.contains(feature) + } else { + !categories.contains(feature) + } + } + override def equals(o: Any): Boolean = { o match { case other: CategoricalSplit => featureIndex == other.featureIndex && @@ -172,6 +186,10 @@ final class ContinuousSplit private[ml] (override val featureIndex: Int, val thr } } + override private[tree] def shouldGoLeft(feature: Double): Boolean = { + feature <= threshold + } + override def equals(o: Any): Boolean = { o match { case other: ContinuousSplit => diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala new file mode 100644 index 000000000000..44a6976cc3cd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -0,0 +1,499 @@ +/* + * 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.ml.tree.impl + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.Logging +import org.apache.spark.annotation.Experimental +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.impl.TreeUtil._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType +import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} +import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy} +import org.apache.spark.mllib.tree.model.{Predict => OldPredict} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.collection.BitSet + + +/** + * DecisionTree which partitions data by feature. + * + * Algorithm: + * - Repartition data, grouping by feature. + * - Prep data (sort continuous features). + * - On each partition, initialize instance--node map with each instance at root node. + * - Iterate, training 1 new level of the tree at a time: + * - On each partition, for each feature on the partition, select the best split for each node. + * - Aggregate best split for each node. + * - Aggregate bit vector (1 bit/instance) indicating whether each instance splits + * left or right. + * - Broadcast bit vector. On each partition, update instance--node map. + * + * TODO: Update to use a sparse column store. + */ +@Experimental +class AltDT (private val strategy: Strategy) extends Serializable with Logging { + + import AltDT.{FeatureVector, PartitionInfo, bitSubvectorFromSplit} + + strategy.assertValid() + + private def getImpurityAggregator(): ImpurityAggregatorSingle = { + strategy.impurity match { + case Entropy => new EntropyAggregatorSingle(strategy.numClasses) + case Gini => new GiniAggregatorSingle(strategy.numClasses) + case Variance => new VarianceAggregatorSingle + } + } + + /** + * Method to train a decision tree model over an RDD. + */ + def train(input: RDD[LabeledPoint], parentUID: Option[String] = None): DecisionTreeModel = { + // TODO: Check validity of params + val rootNode = trainImpl(input) + RandomForest.finalizeTree(rootNode, strategy.algo, parentUID) + } + + def trainImpl(input: RDD[LabeledPoint]): Node = { + // The case with 1 node (depth = 0) is handled separately. + // This allows all iterations in the depth > 0 case to use the same code. + if (strategy.maxDepth == 0) { + val impurityAggregator: ImpurityAggregatorSingle = + input.aggregate(getImpurityAggregator())( + (agg, lp) => agg.update(lp.label, 1.0), + (agg1, agg2) => agg1.add(agg2)) + val impurityCalculator = impurityAggregator.getCalculator + return new LeafNode(impurityCalculator.getPredict.predict, impurityCalculator.calculate()) + } + + // Prepare column store. + // Note: rowToColumnStoreDense checks to make sure numRows < Int.MaxValue. + // TODO: Is this mapping from arrays to iterators to arrays (when constructing learningData)? + // Or is the mapping implicit (i.e., not costly)? + val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) + val numRows: Int = colStoreInit.first()._2.size + val labels = new Array[Double](numRows) + input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => + labels(rowIndex.toInt) = label + } + val labelsBc = input.sparkContext.broadcast(labels) + // NOTE: Labels are not sorted with features since that would require 1 copy per feature, + // rather than 1 copy per worker. This means a lot of random accesses. + // We could improve this by applying first-level sorting (by node) to labels. + + // Sort each column by feature values. + val colStore: RDD[FeatureVector] = colStoreInit.map { case (featureIndex: Int, col: Vector) => + if (strategy.categoricalFeaturesInfo.contains(featureIndex)) { + FeatureVector.fromOriginal(featureIndex, FeatureType.Categorical, col) + } else { + FeatureVector.fromOriginal(featureIndex, FeatureType.Continuous, col) + } + } + // Group columns together into one array of columns per partition. + val groupedColStore: RDD[Array[FeatureVector]] = colStore.mapPartitions { iterator => + val groupedCols = new ArrayBuffer[FeatureVector] + iterator.foreach(groupedCols += _) + Iterator(groupedCols.toArray) + } + groupedColStore.persist(StorageLevel.MEMORY_AND_DISK) + + // Initialize partitions with 1 partition (i.e., each instance at the root node). + var partitionInfos: RDD[PartitionInfo] = groupedColStore.map { groupedCols => + val initActive = new BitSet(1) + initActive.setUntil(1) + new PartitionInfo(groupedCols, Array[Int](0, numRows), initActive) + } + + // Initialize model. + // Note: We do not use node indices. + val rootNode = LearningNode.emptyNode(-1) + // Active nodes (still being split), updated each iteration + var activeNodePeriphery: Array[LearningNode] = Array(rootNode) + var numNodeOffsets: Int = 2 + + // Iteratively learn, one level of the tree at a time. + var currentLevel = 0 + while (currentLevel < strategy.maxDepth) { + + // On each partition, for each feature on the partition, select the best split for each node. + // This will use: + // - groupedColStore (the features) + // - partitionInfos (the node -> instance mapping) + // - labelsBc (the labels column) + // Each worker returns: + // for each active node, best split + info gain + val partBestSplitsAndGains: RDD[Array[(Split, InfoGainStats)]] = partitionInfos.map { + case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => + val localLabels = labelsBc.value + // Iterate over the active nodes in the current level. + activeNodes.iterator.map { nodeIndexInLevel: Int => + val fromOffset = nodeOffsets(nodeIndexInLevel) + val toOffset = nodeOffsets(nodeIndexInLevel + 1) + val splitsAndStats = + columns.map { col => + chooseSplit(col, localLabels, fromOffset, toOffset, getImpurityAggregator()) + } + // nodeIndexInLevel -> splitsAndStats.maxBy(_._2.gain) + splitsAndStats.maxBy(_._2.gain) + }.toArray + } + + // Aggregate best split for each active node. + val bestSplitsAndGains: Array[(Split, InfoGainStats)] = + partBestSplitsAndGains.reduce { case (splitsGains1, splitsGains2) => + splitsGains1.zip(splitsGains2).map { case ((split1, gain1), (split2, gain2)) => + if (gain1.gain >= gain2.gain) { + (split1, gain1) + } else { + (split2, gain2) + } + } + } + + // Update current model and node periphery. + // Note: This flatMap has side effects (on the model). + activeNodePeriphery = bestSplitsAndGains.zipWithIndex.flatMap { + case ((split, stats), nodeIdx) => + val node = activeNodePeriphery(nodeIdx) + node.predictionStats = new OldPredict(stats.prediction, -1) + node.impurity = stats.impurity + if (stats.gain > strategy.getMinInfoGain) { + // TODO: Add prediction probability once that is added properly to trees + node.leftChild = + Some(LearningNode(-1, stats.leftPredict, stats.leftImpurity, isLeaf = false)) + node.rightChild = + Some(LearningNode(-1, stats.rightPredict, stats.rightImpurity, isLeaf = false)) + node.split = Some(split) + node.stats = Some(stats.toOld) + Iterator(node.leftChild.get, node.rightChild.get) + } else { + node.isLeaf = true + Iterator() + } + } + // We keep all old nodeOffsets and add one for each node split. + // Each node split adds 2 nodes to activeNodePeriphery. + numNodeOffsets = numNodeOffsets + activeNodePeriphery.length / 2 + + // TODO: Check to make sure we split something, and stop otherwise. + val doneLearning = currentLevel >= strategy.maxDepth + + if (!doneLearning) { + // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. + // - Send chosen splits to workers. + // - Each worker creates part of the bit vector corresponding to the splits it created. + // - Aggregate the partial bit vectors to create one vector (of length numRows). + // Correction: Aggregate only the pieces of that vector corresponding to instances at + // active nodes. + val bestSplitsBc: Broadcast[Array[Split]] = + input.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) + val workerBitSubvectors: RDD[Array[BitSubvector]] = partitionInfos.map { + case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => + val localBestSplits: Array[Split] = bestSplitsBc.value + // localFeatureIndex[feature index] = index into PartitionInfo.columns + val localFeatureIndex: Map[Int, Int] = columns.map(_.featureIndex).zipWithIndex.toMap + activeNodes.iterator.zip(localBestSplits.iterator).flatMap { + case (nodeIndexInLevel: Int, split: Split) => + if (localFeatureIndex.contains(split.featureIndex)) { + // This partition has the column (feature) used for this split. + val fromOffset = nodeOffsets(nodeIndexInLevel) + val toOffset = nodeOffsets(nodeIndexInLevel + 1) + val colIndex: Int = localFeatureIndex(split.featureIndex) + Iterator(bitSubvectorFromSplit(columns(colIndex), fromOffset, toOffset, split)) + } else { + Iterator() + } + }.toArray + } + val aggBitVectors: Array[BitSubvector] = workerBitSubvectors.reduce(BitSubvector.merge) + bestSplitsBc.unpersist() + + // Broadcast aggregated bit vectors. On each partition, update instance--node map. + val aggBitVectorsBc = input.sparkContext.broadcast(aggBitVectors) + partitionInfos = partitionInfos.map { partitionInfo => + partitionInfo.update(aggBitVectorsBc.value, numNodeOffsets, activeNodePeriphery.length) + } + // TODO: unpersist aggBitVectorsBc after action. + } + + currentLevel += 1 + } + + // Done with learning + groupedColStore.unpersist() + labelsBc.unpersist() + // TODO: return model + rootNode.toNode + } + + // NOTE: The order of col does not correspond to the order of labels. Use the index in col. + /** + * + * @param col + * @param labels + * @param fromOffset + * @param toOffset + * @return + */ + private def chooseSplit( + col: FeatureVector, + labels: Array[Double], + fromOffset: Int, + toOffset: Int, + impurityAgg: ImpurityAggregatorSingle): (Split, InfoGainStats) = { + val featureIndex = col.featureIndex + val valuesForNode = col.values.view.slice(fromOffset, toOffset) + val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) + impurityAgg.clear() + val fullImpurityAgg = impurityAgg.deepCopy() + labels.foreach(fullImpurityAgg.update(_, 1.0)) + col.featureType match { + case FeatureType.Categorical => + chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg) + case FeatureType.Continuous => + chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg) + } + } + + private def chooseCategoricalSplit( + featureIndex: Int, + values: Seq[Double], + labels: Seq[Double], + leftImpurityAgg: ImpurityAggregatorSingle, + rightImpurityAgg: ImpurityAggregatorSingle): (Split, InfoGainStats) = ??? + + /** + * Choose splitting rule: feature value <= threshold + */ + private def chooseContinuousSplit( + featureIndex: Int, + values: Seq[Double], + labels: Seq[Double], + leftImpurityAgg: ImpurityAggregatorSingle, + rightImpurityAgg: ImpurityAggregatorSingle): (Split, InfoGainStats) = { + val prediction = leftImpurityAgg.getCalculator.getPredict + + var bestThreshold: Double = Double.NegativeInfinity + var bestLeftImpurityAgg = leftImpurityAgg.deepCopy() + var bestGain: Double = 0.0 + val fullImpurity = rightImpurityAgg.getCalculator.calculate() + var leftWeight: Double = 0.0 + var rightWeight: Double = rightImpurityAgg.getWeight + values.zip(labels).foreach { case (value, label) => + // Move this instance from right to left side of split. + leftImpurityAgg.update(label, 1.0) + rightImpurityAgg.update(label, -1.0) + leftWeight += 1.0 + rightWeight -= 1.0 + // Check gain + val leftImpurity = leftImpurityAgg.getCalculator.calculate() + val rightImpurity = rightImpurityAgg.getCalculator.calculate() + val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity + if (gain > bestGain) { + bestThreshold = value + leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) + bestGain = gain + } + } + + val leftImpurity = bestLeftImpurityAgg.getCalculator.calculate() + val bestRightImpurityAgg = + leftImpurityAgg.deepCopy().add(rightImpurityAgg).subtract(bestLeftImpurityAgg) + val rightImpurity = bestRightImpurityAgg.getCalculator.calculate() + val bestGainStats = new InfoGainStats(prediction.predict, bestGain, fullImpurity, + leftImpurity, rightImpurity, bestLeftImpurityAgg.getCalculator.getPredict, + bestRightImpurityAgg.getCalculator.getPredict) + (new ContinuousSplit(featureIndex, bestThreshold), bestGainStats) + } + +} + +object AltDT extends Serializable with Logging { + + /** + * Feature vector types are based on (feature type, representation). + * The feature type can be continuous or categorical. + * + * Features are sorted by value, so we must store indices + values. + * These values are currently stored in a dense representation only. + * TODO: Support sparse storage (to optimize deeper levels of the tree), and maybe compressed + * storage (to optimize upper levels of the tree). + */ + private class FeatureVector( + val featureIndex: Int, + val featureType: FeatureType, + val values: Array[Double], + val indices: Array[Int]) + extends Serializable { + + def deepCopy(): FeatureVector = + new FeatureVector(featureIndex, featureType, values.clone(), indices.clone()) + } + + private object FeatureVector { + /** Store column sorted by feature values. */ + def fromOriginal( + featureIndex: Int, + featureType: FeatureType, + featureVector: Vector): FeatureVector = { + val (values, indices) = featureVector.toArray.zipWithIndex.sorted.unzip + new FeatureVector(featureIndex, featureType, values.toArray, indices.toArray) + } + } + + /** + * For a given feature, for a given node, apply a split and return a bit vector indicating the + * outcome of the split for each instance at that node. + * + * @param col Column for feature + * @param fromOffset Start offset in col for the node + * @param toOffset End offset in col for the node + * @param split Split to apply to instances at this node. + * @return Bits indicating splits for instances at this node. + * These bits are sorted by the row indices, in order to guarantee an ordering + * understood by all workers. + * Thus, the bit indices used are based on 2-level sorting: first by node, and + * second by sorted row indices within the node's rows. + * bit[index in sorted array of row indices] = false for left, true for right + */ + def bitSubvectorFromSplit( + col: FeatureVector, + fromOffset: Int, + toOffset: Int, + split: Split): BitSubvector = { + val nodeRowIndices = col.indices.view.slice(fromOffset, toOffset) + val nodeRowValues = col.values.view.slice(fromOffset, toOffset) + val nodeRowValuesSortedByIndices = nodeRowIndices.zip(nodeRowValues).sortBy(_._1).map(_._2) + val bitv = new BitSubvector(fromOffset, toOffset) + nodeRowValuesSortedByIndices.zipWithIndex.foreach { case (value, i) => + if (!split.shouldGoLeft(value)) bitv.set(fromOffset + i) + } + bitv + } + + /** + * Intermediate data stored on each partition during learning. + * + * Node indexing for nodeOffsets, activeNodes: + * Nodes are indexed left-to-right along the periphery of the tree, with 0-based indices. + * The periphery is the set of leaf nodes (active and inactive). + * + * @param columns Subset of columns (features) stored in this partition. + * Each column is sorted first by nodes (left-to-right along the tree periphery); + * all columns share this first level of sorting. + * Within each node's group, each column is sorted based on feature value; + * this second level of sorting differs across columns. + * @param nodeOffsets Offsets into the columns indicating the first level of sorting (by node). + * The rows corresponding to node i are in the range + * [nodeOffsets(i), nodeOffsets(i+1)). + * @param activeNodes Nodes which are active (still being split). + * Inactive nodes are known to be leafs in the final tree. + * TODO: Should this (and even nodeOffsets) not be stored in PartitionInfo, + * but instead on the driver? + */ + private case class PartitionInfo( + columns: Array[FeatureVector], + nodeOffsets: Array[Int], + activeNodes: BitSet) + extends Serializable { + + /** + * Update columns and nodeOffsets for the next level of the tree. + * + * Update columns: + * For each column, + * For each (previously) active node, + * Sort corresponding range of instances based on bit vector. + * Update nodeOffsets, activeNodes: + * Split offsets for nodes which split (which can be identified using the bit vector). + * + * @param bitVectors Bit vectors encoding splits for the next level of the tree. + * These must follow a 2-level ordering, where the first level is by node + * and the second level is by row index. + * bitVector(i) = false iff instance i goes to the left child. + * For instances at inactive (leaf) nodes, the value can be arbitrary. + * @return Updated partition info + */ + def update(bitVectors: Array[BitSubvector], newNumNodeOffsets: Int, numNewActiveNodes: Int): PartitionInfo = { + val newColumns = columns.map { oldCol => + val col = oldCol.deepCopy() + var curBitVecIdx = 0 + activeNodes.iterator.foreach { nodeIdx => + val from = nodeOffsets(nodeIdx) + val to = nodeOffsets(nodeIdx + 1) + if (bitVectors(curBitVecIdx).to <= from) curBitVecIdx += 1 + val curBitVector = bitVectors(curBitVecIdx) + // Sort range [from, to) based on indices. This is required to match the bit vector + // across all workers. See [[bitSubvectorFromSplit]] for details. + val sortedRange = + col.indices.view.slice(from, to).zip(col.values.view.slice(from, to)).sortBy(_._1) + // Sort range [from, to) based on bit vector. + sortedRange.zipWithIndex.map { case ((idx, value), i) => + val bit = curBitVector.get(from + i) + // TODO: In-place merge, rather than general sort. + (bit, value, idx) + }.sorted.zipWithIndex.foreach { case ((bit, value, idx), i) => + col.values(from + i) = value + col.indices(from + i) = idx + } + } + col + } + + // Create a 2-level representation of the new nodeOffsets (to be flattened). + val newNodeOffsets = nodeOffsets.map(Array(_)) + var curBitVecIdx = 0 + activeNodes.iterator.foreach { nodeIdx => + val from = nodeOffsets(nodeIdx) + val to = nodeOffsets(nodeIdx + 1) + if (bitVectors(curBitVecIdx).to <= from) curBitVecIdx += 1 + val curBitVector = bitVectors(curBitVecIdx) + // Count number of values splitting to left vs. right + val numRight = Range(from, to).count(curBitVector.get) + val numLeft = to - from - numRight + if (numRight != 0 && numRight != 0) { + // node is split + val oldOffset = newNodeOffsets(nodeIdx + 1).head + newNodeOffsets(nodeIdx + 1) = Array(oldOffset, oldOffset + numLeft) + } + } + + // Identify the new activeNodes based on the 2-level representation of the new nodeOffsets. + val newActiveNodes = new BitSet(newNumNodeOffsets) + var newNodeOffsetsIdx = 0 + newNodeOffsets.foreach { offsets => + if (offsets.length == 2) { + newActiveNodes.set(newNodeOffsetsIdx) + newActiveNodes.set(newNodeOffsetsIdx + 1) + newNodeOffsetsIdx += 2 + } else { + newNodeOffsetsIdx += 1 + } + } + + new PartitionInfo(newColumns, newNodeOffsets.flatten, newActiveNodes) + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala new file mode 100644 index 000000000000..d251fa43e77d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -0,0 +1,77 @@ +/* + * 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.ml.tree.impl + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.util.collection.BitSet + + +private class BitSubvector(val from: Int, val to: Int) extends Serializable { + + val numBits: Int = to - from + + /** Element i will be put at location i + offset in the BitSet */ + private val offset: Int = 64 - (numBits % 64) + + private val bits: BitSet = new BitSet(numBits + offset) + + def set(bit: Int): Unit = bits.set(bit + offset - from) + + def get(bit: Int): Boolean = bits.get(bit + offset - from) + + /** Get an iterator over the set bits. */ + def iterator: Iterator[Int] = new Iterator[Int] { + val iter = bits.iterator + override def hasNext: Boolean = iter.hasNext + override def next(): Int = iter.next() - offset + from + } +} + +private object BitSubvector { + + def merge(parts1: Array[BitSubvector], parts2: Array[BitSubvector]): Array[BitSubvector] = { + // Merge sorted parts1, parts2 + val sortedSubvectors = (parts1 ++ parts2).sortBy(_.from) + // Merge adjacent PartialBitVectors (for adjacent node ranges) + val newSubvectorRanges: Array[(Int, Int)] = { + val newSubvRanges = ArrayBuffer.empty[(Int, Int)] + var i = 1 + var currentFrom = sortedSubvectors.head.from + while (i < sortedSubvectors.length) { + if (sortedSubvectors(i - 1).to != sortedSubvectors(i).from) { + newSubvRanges.append((currentFrom, sortedSubvectors(i - 1).to)) + currentFrom = sortedSubvectors(i).from + } + i += 1 + } + newSubvRanges.append((currentFrom, sortedSubvectors.last.to)) + newSubvRanges.toArray + } + val newSubvectors = newSubvectorRanges.map { case (from, to) => new BitSubvector(from, to) } + var curNewSubvIdx = 0 + sortedSubvectors.foreach { subv => + if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 + val newSubv = newSubvectors(curNewSubvIdx) + // TODO: More efficient (word-level) copy. + subv.iterator.foreach(idx => newSubv.set(idx)) + } + assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check + newSubvectors + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/InfoGainStats.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/InfoGainStats.scala new file mode 100644 index 000000000000..7520ac225fde --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/InfoGainStats.scala @@ -0,0 +1,70 @@ +/* + * 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.ml.tree.impl + +import org.apache.spark.mllib.tree.model.{InformationGainStats => OldInformationGainStats, Predict} + +/** + * Information gain statistics for each split + * @param gain information gain value + * @param impurity current node impurity + * @param leftImpurity left node impurity + * @param rightImpurity right node impurity + * @param leftPredict left node predict + * @param rightPredict right node predict + */ +private[tree] class InfoGainStats( + val prediction: Double, + val gain: Double, + val impurity: Double, + val leftImpurity: Double, + val rightImpurity: Double, + val leftPredict: Predict, + val rightPredict: Predict) extends Serializable { + + override def toString: String = { + s"prediction = $prediction, gain = $gain, impurity = $impurity, " + + s"left impurity = $leftImpurity, right impurity = $rightImpurity" + } + + override def equals(o: Any): Boolean = o match { + case other: InfoGainStats => + prediction == other.prediction && + gain == other.gain && + impurity == other.impurity && + leftImpurity == other.leftImpurity && + rightImpurity == other.rightImpurity && + leftPredict == other.leftPredict && + rightPredict == other.rightPredict + case _ => false + } + + override def hashCode: Int = { + com.google.common.base.Objects.hashCode( + prediction: java.lang.Double, + gain: java.lang.Double, + impurity: java.lang.Double, + leftImpurity: java.lang.Double, + rightImpurity: java.lang.Double, + leftPredict, + rightPredict) + } + + def toOld: OldInformationGainStats = new OldInformationGainStats(gain, impurity, leftImpurity, + rightImpurity, leftPredict, rightPredict) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 4ac51a475474..8b5128676d58 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -179,6 +179,13 @@ private[ml] object RandomForest extends Logging { } } + topNodes.map(finalizeTree(_, strategy.algo, parentUID)) + } + + private[tree] def finalizeTree( + rootNode: Node, + algo: OldAlgo.Algo, + parentUID: Option[String]): DecisionTreeModel = { parentUID match { case Some(uid) => if (strategy.algo == OldAlgo.Classification) { @@ -186,7 +193,7 @@ private[ml] object RandomForest extends Logging { new DecisionTreeClassificationModel(uid, rootNode.toNode, strategy.getNumClasses) } } else { - topNodes.map(rootNode => new DecisionTreeRegressionModel(uid, rootNode.toNode)) + new DecisionTreeRegressionModel(uid, rootNode) } case None => if (strategy.algo == OldAlgo.Classification) { @@ -194,7 +201,7 @@ private[ml] object RandomForest extends Logging { new DecisionTreeClassificationModel(rootNode.toNode, strategy.getNumClasses) } } else { - topNodes.map(rootNode => new DecisionTreeRegressionModel(rootNode.toNode)) + new DecisionTreeRegressionModel(rootNode) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreeUtil.scala similarity index 96% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreeUtil.scala index 9fcc1a4fb208..fa6eb63685d2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/Util.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreeUtil.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import scala.collection.mutable.ArrayBuffer -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vectors, Vector} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.rdd.RDD -private[tree] object Util { +private[tree] object TreeUtil { /** * Convert a dataset of [[Vector]] from row storage to column storage. @@ -61,7 +60,7 @@ private[tree] object Util { return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) } val numCols = rowStore.take(1)(0).size - val numSourcePartitions = rowStore.partitions.size + val numSourcePartitions = rowStore.partitions.length val numTargetPartitions = Math.min(numCols, numSourcePartitions) if (numTargetPartitions == 0) { return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) @@ -125,7 +124,7 @@ private[tree] object Util { iterator.foreach { case (sourcePartitionIndex, partCols) => collectedPartCols(sourcePartitionIndex) = partCols } - val rowOffsets: Array[Int] = collectedPartCols.map(_(0).size).scanLeft(0)(_ + _) + val rowOffsets: Array[Int] = collectedPartCols.map(_(0).length).scanLeft(0)(_ + _) val numRows = rowOffsets.last // Initialize full columns val fromCol = groupIndex * maxColumnsPerPartition @@ -161,7 +160,7 @@ private[tree] object Util { */ private def countNonZerosPerColumn(rowStore: RDD[Vector]): Array[Long] = { val firstRow = rowStore.take(1) - if (firstRow.size == 0) { + if (firstRow.length == 0) { return Array.empty[Long] } val numCols = firstRow(0).size @@ -176,7 +175,7 @@ private[tree] object Util { } case sv: SparseVector => var k = 0 - while (k < sv.indices.size) { + while (k < sv.indices.length) { if (sv.values(k) != 0.0) partColSizes(sv.indices(k)) += 1 k += 1 } @@ -222,8 +221,8 @@ private[tree] object Util { // Compute the number of non-zeros in each column. val colSizes: Array[Long] = countNonZerosPerColumn(rowStore) - val numCols = colSizes.size - val numSourcePartitions = rowStore.partitions.size + val numCols = colSizes.length + val numSourcePartitions = rowStore.partitions.length if (numCols == 0 || numSourcePartitions == 0) { return rowStore.sparkContext.parallelize(Seq.empty[(Int, Vector)]) } @@ -254,7 +253,7 @@ private[tree] object Util { startCols += numCols startCols.toArray } - val numGroups = groupStartColumns.size - 1 // actual number of destination partitions + val numGroups = groupStartColumns.length - 1 // actual number of destination partitions /* On each partition, re-organize into groups of columns: (groupIndex, (sourcePartitionIndex, partCols)), @@ -303,7 +302,7 @@ private[tree] object Util { */ var groupIndex = 0 var k = 0 // index into SparseVector non-zeros - val nnz = sv.indices.size + val nnz = sv.indices.length while (groupIndex < numGroups && k < nnz) { val fromColumn = groupStartColumns(groupIndex) val groupEndColumn = groupStartColumns(groupIndex + 1) @@ -347,7 +346,7 @@ private[tree] object Util { iterator.foreach { case (sourcePartitionIndex, partCols) => collectedPartCols(sourcePartitionIndex) = partCols partCols.zipWithIndex.foreach { case (partCol, colIdx) => - nzCounts(colIdx)(sourcePartitionIndex) += partCol.indices.size + nzCounts(colIdx)(sourcePartitionIndex) += partCol.indices.length } } // nzOffsets(colIdx)(sourcePartitionIndex) = cumulative number of non-zeros diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala new file mode 100644 index 000000000000..4d4b52b62939 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala @@ -0,0 +1,152 @@ +/* + * 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.ml.tree + +import org.apache.spark.mllib.tree.impurity.{EntropyCalculator, GiniCalculator, ImpurityCalculator, + VarianceCalculator} + +/** + * Version of impurity aggregator which owns its data and is only for 1 node. + */ +private[tree] abstract class ImpurityAggregatorSingle(val stats: Array[Double]) + extends Serializable { + + def statsSize: Int = stats.length + + /** + * Add two aggregators: this + other + * @return This aggregator (modified). + */ + def add(other: ImpurityAggregatorSingle): this.type = { + var i = 0 + while (i < statsSize) { + stats(i) += other.stats(i) + i += 1 + } + this + } + + /** + * Subtract another aggregators from this one: this - other + * @return This aggregator (modified). + */ + def subtract(other: ImpurityAggregatorSingle): this.type = { + var i = 0 + while (i < statsSize) { + stats(i) -= other.stats(i) + i += 1 + } + this + } + + /** + * Update stats with the given label. + * @return This aggregator (modified). + */ + def update(label: Double, instanceWeight: Double): this.type + + /** Get an [[ImpurityCalculator]] for the current stats. */ + def getCalculator: ImpurityCalculator + + def deepCopy(): ImpurityAggregatorSingle + + /** Total weight of instances in this aggregator */ + def getWeight: Double + + /** Resets this aggregator as though nothing has been added to it. */ + def clear(): this.type = { + var i = 0 + while (i < statsSize) { + stats(i) = 0.0 + i += 1 + } + this + } +} + +/** + * Version of Entropy aggregator which owns its data and is only for one node. + */ +private[tree] class EntropyAggregatorSingle private (stats: Array[Double]) + extends ImpurityAggregatorSingle(stats) with Serializable { + + def this(numClasses: Int) = this(new Array[Double](numClasses)) + + def update(label: Double, instanceWeight: Double): this.type = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"EntropyAggregatorSingle given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + stats(label.toInt) += instanceWeight + this + } + + def getCalculator: EntropyCalculator = new EntropyCalculator(stats) + + override def deepCopy(): ImpurityAggregatorSingle = new EntropyAggregatorSingle(stats.clone()) + + override def getWeight: Double = stats.sum +} + +/** + * Version of Gini aggregator which owns its data and is only for one node. + */ +private[tree] class GiniAggregatorSingle private (stats: Array[Double]) + extends ImpurityAggregatorSingle(stats) with Serializable { + + def this(numClasses: Int) = this(new Array[Double](numClasses)) + + def update(label: Double, instanceWeight: Double): this.type = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"GiniAggregatorSingle given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + stats(label.toInt) += instanceWeight + this + } + + def getCalculator: GiniCalculator = new GiniCalculator(stats) + + override def deepCopy(): ImpurityAggregatorSingle = new GiniAggregatorSingle(stats.clone()) + + override def getWeight: Double = stats.sum +} + +/** + * Version of Variance aggregator which owns its data and is only for one node. + */ +private[tree] class VarianceAggregatorSingle + extends ImpurityAggregatorSingle(new Array[Double](3)) with Serializable { + + def update(label: Double, instanceWeight: Double): this.type = { + stats(0) += instanceWeight + stats(1) += instanceWeight * label + stats(2) += instanceWeight * label * label + this + } + + def getCalculator: VarianceCalculator = new VarianceCalculator(stats) + + override def deepCopy(): ImpurityAggregatorSingle = { + val tmp = new VarianceAggregatorSingle() + stats.copyToArray(tmp.stats) + tmp + } + + override def getWeight: Double = stats(0) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala deleted file mode 100644 index 2c6fa09acbf0..000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/AltDT.scala +++ /dev/null @@ -1,352 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.tree - -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType -import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType -import org.apache.spark.mllib.tree.impurity._ -import org.apache.spark.storage.StorageLevel - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.Logging -import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vectors, Vector} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} -import org.apache.spark.mllib.tree.impl.Util._ -import org.apache.spark.mllib.tree.model._ -import org.apache.spark.rdd.RDD - - -/** - * DecisionTree which partitions data by feature. - * - * Algorithm: - * - Repartition data, grouping by feature. - * - Prep data (sort continuous features). - * - On each partition, initialize instance--node map with each instance at root node. - * - Iterate, training 1 new level of the tree at a time: - * - On each partition, for each feature on the partition, select the best split for each node. - * - Aggregate best split for each node. - * - Aggregate bit vector (1 bit/instance) indicating whether each instance splits - * left or right. - * - Broadcast bit vector. On each partition, update instance--node map. - * - * TODO: Update to use a sparse column store. - */ -@Experimental -class AltDT (private val strategy: Strategy) extends Serializable with Logging { - - strategy.assertValid() - - /** - * Feature vector types are based on (feature type, representation). - * The feature type can be continuous or categorical. - * - * Features are sorted by value, so we must store indices + values. - * These values are currently stored in a dense representation only. - * TODO: Support sparse storage. - */ - private class FeatureVector( - val featureIndex: Int, - val featureType: FeatureType, - val values: Array[Double], - val indices: Array[Int]) - extends Serializable - - private object FeatureVector { - /** Store column sorted by feature values. */ - def fromOriginal( - featureIndex: Int, - featureType: FeatureType, - featureVector: Vector): FeatureVector = { - val (values, indices) = featureVector.toArray.zipWithIndex.sorted.unzip - new FeatureVector(featureIndex, featureType, values.toArray, indices.toArray) - } - } - - /** - * Intermediate data stored on each partition during learning. - * @param columns Subset of columns (features) stored in this partition. - * @param nodeOffsets Offsets into the rows (instances) indicating how they are split among - * nodes. The rows corresponding to node i are in the range - * [nodeOffsets(i), nodeOffsets(i+1)). - */ - private case class PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int]) - extends Serializable { - - /** - * Update columns and nodeOffsets for the next level of the tree.. - * @param bitVector Bit vector encoding splits for the next level of the tree. - * bitVector(i) = false iff instance i goes to the left child. - * @return Updated partition info - */ - def update(bitVector: Array[Boolean]): PartitionInfo = ??? - } - - private def getImpurityAggregator: ImpurityAggregatorSingle = { - strategy.impurity match { - case Entropy => new EntropyAggregatorSingle(strategy.numClasses) - case Gini => new GiniAggregatorSingle(strategy.numClasses) - case Variance => new VarianceAggregatorSingle - } - } - - /* - private class NodeIndex private (val splits: Array[Boolean]) extends Serializable { - import NodeIndex._ - - def leftChild: Array[Boolean] = splits :+ LEFT - - def rightChild: Array[Boolean] = splits :+ RIGHT - - } - - private object NodeIndex extends Serializable { - final val LEFT: Boolean = false - final val RIGHT: Boolean = true - def root: NodeIndex = new NodeIndex(Array.emptyBooleanArray) - } - - private class SplitInfo() extends Serializable - */ - - /** - * Method to train a decision tree model over an RDD - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @return DecisionTreeModel that can be used for prediction - */ - def train(input: RDD[LabeledPoint]): DecisionTreeModel = { - // TODO: Check validity of params - - // The case with 1 node (depth = 0) is handled separately. - // This allows all iterations in the depth > 0 case to use the same code. - if (strategy.maxDepth == 0) { - val impurityAggregator: ImpurityAggregatorSingle = - input.aggregate(getImpurityAggregator)( - (agg, lp) => agg.update(lp.label, 1.0), - (agg1, agg2) => agg1.merge(agg2)) - val impurityCalculator = impurityAggregator.getCalculator - val node = Node(nodeIndex = 0, predict = impurityCalculator.getPredict, - impurity = impurityCalculator.calculate(), isLeaf = true) - return new DecisionTreeModel(node, strategy.algo) - } - - // Prepare column store. - // Note: rowToColumnStoreDense checks to make sure numRows < Int.MaxValue. - // TODO: Is this mapping from arrays to iterators to arrays (when constructing learningData)? - // Or is the mapping implicit (i.e., not costly)? - val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) - val numRows: Int = colStoreInit.take(1)(0)._2.size - val labels = new Array[Double](numRows) - input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => - labels(rowIndex.toInt) = label - } - val labelsBroadcast = input.sparkContext.broadcast(labels) - - // Sort each column by feature values. - val colStore: RDD[FeatureVector] = colStoreInit.map { case (featureIndex: Int, col: Vector) => - if (strategy.categoricalFeaturesInfo.contains(featureIndex)) { - FeatureVector.fromOriginal(featureIndex, FeatureType.Categorical, col) - } else { - FeatureVector.fromOriginal(featureIndex, FeatureType.Continuous, col) - } - } - // Group columns together into one array of columns per partition. - val groupedColStore: RDD[Array[FeatureVector]] = colStore.mapPartitions { iterator => - val groupedCols = new ArrayBuffer[FeatureVector] - iterator.foreach(groupedCols += _) - Iterator(groupedCols.toArray) - } - groupedColStore.persist(StorageLevel.MEMORY_AND_DISK) - - // Initialize partitions with 1 partition (i.e., each instance at the root node). - var partitionInfos: RDD[PartitionInfo] = groupedColStore.map { groupedCols => - new PartitionInfo(groupedCols, Array[Int](0, numRows)) - } - - // Iteratively learn, one level of the tree at a time. - var currentLevel = 0 - while (currentLevel < strategy.maxDepth) { - - val numNodesInLevel: Int = ??? - - // On each partition, for each feature on the partition, select the best split for each node. - // This will use: - // - groupedColStore (the features) - // - partitionInfos (the node -> instance mapping) - // - labelsBroadcast (the labels column) - // Each worker returns: - // for each node, best split + info gain - val workerBestSplitsAndGains: RDD[Array[(Split, InformationGainStats)]] = partitionInfos.map { - case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int]) => - val localLabels = labelsBroadcast.value - val localBestSplitsAndGains = new Array[(Split, InformationGainStats)](numNodesInLevel) - // Iterate over the nodes in the current level. - var nodeIndexInLevel = 0 - while (nodeIndexInLevel < numNodesInLevel) { - val fromOffset = nodeOffsets(nodeIndexInLevel) - val toOffset = nodeOffsets(nodeIndexInLevel + 1) - val splitsAndStats = - columns.map(col => chooseSplit(col, localLabels, fromOffset, toOffset)) - localBestSplitsAndGains(nodeIndexInLevel) = splitsAndStats.maxBy(_._2.gain) - nodeIndexInLevel += 1 - } - localBestSplitsAndGains - } - - // Aggregate best split for each node. - // TODO: verify that initialization gives info gain 0 - val bestSplitsAndGains: Array[(Split, InformationGainStats)] = - workerBestSplitsAndGains.fold(new Array[(Split, InformationGainStats)](numNodesInLevel)){ - case (splitsGains1, splitsGains2) => - splitsGains1.zip(splitsGains2).map { case ((split1, gain1), (split2, gain2)) => - if (gain1.gain >= gain2.gain) { - (split1, gain1) - } else { - (split2, gain2) - } - } - } - - // Update current model. - // TODO - - // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left or right. - // - Send chosen splits to workers. - // - Each worker creates part of the bit vector corresponding to the splits it created. - // - Aggregate the partial bit vectors to create one vector (of length numRows). - val broadcastBestSplits: Broadcast[Array[Split]] = - input.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) - val workerPartialBitVectors: RDD[Array[PartialBitVector]] = partitionInfos.map { - case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int]) => - val localBestSplits = broadcastBestSplits.value - // localFeatureIndex[feature index] = index into groupedCols - val localFeatureIndex: Map[Int, Int] = columns.map(_.featureIndex).zipWithIndex.toMap - localBestSplits.zipWithIndex.flatMap { case (split: Split, nodeIndexInLevel: Int) => - if (localFeatureIndex.contains(split.feature)) { - val fromOffset = nodeOffsets(nodeIndexInLevel) - val toOffset = nodeOffsets(nodeIndexInLevel + 1) - val colIndex: Int = localFeatureIndex(split.feature) - Iterator(PartialBitVector.fromSplit( - columns(colIndex), fromOffset, toOffset, split, nodeIndexInLevel)) - } else { - Iterator() - } - }.toArray - } - val fullBitVector: Array[Boolean] = { - val aggBitVector: Array[PartialBitVector] = - workerPartialBitVectors.fold(Array.empty[PartialBitVector])(PartialBitVector.merge) - assert(aggBitVector.size == 1) - assert(aggBitVector(0).fromNode == 0 && aggBitVector(0).toNode == numNodesInLevel) - aggBitVector(0).bits - } - - // Broadcast bit vector. On each partition, update instance--node map. - val broadcastBitVector = input.sparkContext.broadcast(fullBitVector) - partitionInfos = partitionInfos.map { - case partitionInfo: PartitionInfo => - val localBitVector: Array[Boolean] = broadcastBitVector.value - partitionInfo.update(localBitVector) - } - - currentLevel += 1 - } - - // Done with learning - groupedColStore.unpersist() - labelsBroadcast.unpersist() - // TODO: return model - } - - // TODO - // NOTE: The order of col does not correspond to the order of labels. Use the index in col. - private def chooseSplit( - col: FeatureVector, - labels: Array[Double], - fromOffset: Int, - toOffset: Int): (Split, InformationGainStats) = ??? - - private class PartialBitVector(val fromNode: Int, val toNode: Int, val bits: Array[Boolean]) - - private object PartialBitVector { - def merge( - parts1: Array[PartialBitVector], - parts2: Array[PartialBitVector]): Array[PartialBitVector] = { - // Merge sorted parts1, parts2 - val sortedParts = Array.fill[PartialBitVector](parts1.size + parts2.size)(null) - var i1 = 0 // indexing parts1 - var i2 = 0 // indexing parts2 - while (i1 < parts1.size || i2 < parts2.size) { - val i = i1 + i2 // index into sortedParts - if (i1 < parts1.size) { - if (i2 < parts2.size) { - // Choose between parts1,2 - if (parts1(i1).fromNode < parts2(i2).fromNode) { - sortedParts(i) = parts1(i1) - i1 += 1 - } else { - sortedParts(i) = parts2(i2) - i2 += 1 - } - } else { - // Take remaining items from parts1 - parts1.view.slice(i1, parts1.size).copyToArray(sortedParts, i) - i1 = parts1.size - } - } else { - // Take remaining items from parts2 - parts2.view.slice(i2, parts2.size).copyToArray(sortedParts, i) - i2 = parts2.size - } - } - // Merge adjacent PartialBitVectors (for adjacent node ranges) - val newParts = new ArrayBuffer[PartialBitVector]() - var j = 0 // indexing sortedParts - if (sortedParts.size > 0) { - newParts += sortedParts(0) - j += 1 - } - while (j < sortedParts.size) { - // Check to see if the next PartialBitVector can be merged with the previous one. - if (newParts.last.toNode == sortedParts(j).fromNode) { - // TODO: RIGHT HERE NOW - } else { - - } - j += 1 - } - newParts.toArray - } - - def fromSplit( - col: FeatureVector, - fromOffset: Int, - toOffset: Int, - split: Split, - nodeIndexInLevel: Int) = ??? - } - -} - -object AltDT extends Serializable with Logging { -} 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 b74e3f1f4652..d8769643329b 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 @@ -136,7 +136,7 @@ class Strategy ( * Check validity of parameters. * Throws exception if invalid. */ - private[tree] def assertValid(): Unit = { + private[spark] def assertValid(): Unit = { algo match { case Classification => require(numClasses >= 2, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 4e06380049de..73df6b054a8c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -116,27 +116,6 @@ private[tree] class EntropyAggregator(numClasses: Int) } -/** - * Version of Entropy aggregator which owns its data and is only for one node. - */ -private[tree] class EntropyAggregatorSingle private (stats: Array[Double]) - extends ImpurityAggregatorSingle(stats) with Serializable { - - def this(numClasses: Int) = this(new Array[Double](numClasses)) - - def update(label: Double, instanceWeight: Double): this.type = { - if (label >= statsSize) { - throw new IllegalArgumentException(s"EntropyAggregatorSingle given label $label" + - s" but requires label < numClasses (= $statsSize).") - } - stats(label.toInt) += instanceWeight - this - } - - def getCalculator: EntropyCalculator = new EntropyCalculator(stats) - -} - /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[EntropyAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index f886bfc63515..f21845b21a80 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -112,27 +112,6 @@ private[tree] class GiniAggregator(numClasses: Int) } -/** - * Version of Gini aggregator which owns its data and is only for one node. - */ -private[tree] class GiniAggregatorSingle private (stats: Array[Double]) - extends ImpurityAggregatorSingle(stats) with Serializable { - - def this(numClasses: Int) = this(new Array[Double](numClasses)) - - def update(label: Double, instanceWeight: Double): this.type = { - if (label >= statsSize) { - throw new IllegalArgumentException(s"GiniAggregatorSingle given label $label" + - s" but requires label < numClasses (= $statsSize).") - } - stats(label.toInt) += instanceWeight - this - } - - def getCalculator: GiniCalculator = new GiniCalculator(stats) - -} - /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[GiniAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 4f821406ad17..b9d0f73c85c9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -93,37 +93,6 @@ private[spark] abstract class ImpurityAggregator(val statsSize: Int) extends Ser } -/** - * Version of impurity aggregator which owns its data and is only for 1 node. - */ -private[tree] abstract class ImpurityAggregatorSingle(val stats: Array[Double]) - extends Serializable { - - def statsSize: Int = stats.size - - /** - * Merge two aggregators. - * @return This aggregator (modified). - */ - def merge(other: ImpurityAggregatorSingle): this.type = { - var i = 0 - while (i < statsSize) { - stats(i) += other.stats(i) - i += 1 - } - this - } - - /** - * Update stats with the given label. - * @return This aggregator (modified). - */ - def update(label: Double, instanceWeight: Double): this.type - - /** Get an [[ImpurityCalculator]] for the current stats. */ - def getCalculator: ImpurityCalculator -} - /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[ImpurityAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index d6d7b65fb241..a74197278d6f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -96,23 +96,6 @@ private[tree] class VarianceAggregator() } -/** - * Version of Variance aggregator which owns its data and is only for one node. - */ -private[tree] class VarianceAggregatorSingle - extends ImpurityAggregatorSingle(new Array[Double](3)) with Serializable { - - def update(label: Double, instanceWeight: Double): this.type = { - stats(0) += instanceWeight - stats(1) += instanceWeight * label - stats(2) += instanceWeight * label * label - this - } - - def getCalculator: VarianceCalculator = new VarianceCalculator(stats) - -} - /** * Stores statistics for one (node, feature, bin) for calculating impurity. * Unlike [[GiniAggregator]], this class stores its own data and is for a specific diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala similarity index 94% rename from mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala rename to mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala index de6f312f7d83..cd6e14f02b80 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/TreeUtilSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import scala.collection.mutable -import org.scalatest.FunSuite - -import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors} -import org.apache.spark.mllib.tree.impl.Util._ -import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.ml.tree.impl.TreeUtil._ +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} /** - * Test suite for [[org.apache.spark.mllib.tree.impl.Util]]. + * Test suite for [[TreeUtil]]. */ class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { From 9e7b0e9bc93263343a9d3a136daecf2919026766 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 23 Jul 2015 22:37:29 -0700 Subject: [PATCH 08/34] debugging AltDT, not done yet --- .../examples/ml/DecisionTreeExample.scala | 12 ++- .../DecisionTreeClassifier.scala | 38 +++++++-- .../ml/regression/DecisionTreeRegressor.scala | 37 +++++++-- .../org/apache/spark/ml/tree/AltDTModel.scala | 26 ------ .../org/apache/spark/ml/tree/impl/AltDT.scala | 82 +++++++++++-------- .../spark/ml/tree/impl/BitSubvector.scala | 4 +- .../spark/ml/tree/impl/RandomForest.scala | 2 +- .../org/apache/spark/ml/tree/impurities.scala | 10 +-- 8 files changed, 130 insertions(+), 81 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index f28671f7869f..ca29c9a980d7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -60,6 +60,7 @@ object DecisionTreeExample { testInput: String = "", dataFormat: String = "libsvm", algo: String = "Classification", + algorithm: String = "byRow", maxDepth: Int = 5, maxBins: Int = 32, minInstancesPerNode: Int = 1, @@ -77,6 +78,9 @@ object DecisionTreeExample { opt[String]("algo") .text(s"algorithm (classification, regression), default: ${defaultParams.algo}") .action((x, c) => c.copy(algo = x)) + opt[String]("algorithm") + .text(s"algorithm (byRow, byCol), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algorithm = x)) opt[Int]("maxDepth") .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") .action((x, c) => c.copy(maxDepth = x)) @@ -236,16 +240,18 @@ object DecisionTreeExample { } // (2) Identify categorical features using VectorIndexer. // Features with more than maxCategories values will be treated as continuous. + /* val featuresIndexer = new VectorIndexer() .setInputCol("features") .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer + */ // (3) Learn Decision Tree val dt = algo match { case "classification" => new DecisionTreeClassifier() - .setFeaturesCol("indexedFeatures") + .setFeaturesCol("features") // indexedFeatures .setLabelCol(labelColName) .setMaxDepth(params.maxDepth) .setMaxBins(params.maxBins) @@ -253,9 +259,10 @@ object DecisionTreeExample { .setMinInfoGain(params.minInfoGain) .setCacheNodeIds(params.cacheNodeIds) .setCheckpointInterval(params.checkpointInterval) + .setAlgorithm(params.algorithm) case "regression" => new DecisionTreeRegressor() - .setFeaturesCol("indexedFeatures") + .setFeaturesCol("features") // indexedFeatures .setLabelCol(labelColName) .setMaxDepth(params.maxDepth) .setMaxBins(params.maxBins) @@ -263,6 +270,7 @@ object DecisionTreeExample { .setMinInfoGain(params.minInfoGain) .setCacheNodeIds(params.cacheNodeIds) .setCheckpointInterval(params.checkpointInterval) + .setAlgorithm(params.algorithm) case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") } stages += dt diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 6f70b96b17ec..0aab2cf257b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -18,9 +18,10 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.tree.{DecisionTreeModel, DecisionTreeParams, Node, TreeClassifierParams} -import org.apache.spark.ml.tree.impl.RandomForest +import org.apache.spark.ml.tree.impl.{AltDT, RandomForest} import org.apache.spark.ml.util.{Identifiable, MetadataUtils} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint @@ -62,6 +63,25 @@ final class DecisionTreeClassifier(override val uid: String) override def setImpurity(value: String): this.type = super.setImpurity(value) + /** + * Algorithm used for learning. + * Supported: "byRow" or "byCol" (case sensitive). + * (default = "byRow") + * @group param + */ + val algorithm: Param[String] = new Param[String](this, "algorithm", "Algorithm used " + + "for learning. Supported options:" + + s" ${DecisionTreeClassifier.supportedAlgorithms.mkString(", ")}", + (value: String) => DecisionTreeClassifier.supportedAlgorithms.contains(value)) + + setDefault(algorithm -> "byRow") + + /** @group setParam */ + def setAlgorithm(value: String): this.type = set(algorithm, value) + + /** @group getParam */ + def getAlgorithm: String = $(algorithm) + override protected def train(dataset: DataFrame): DecisionTreeClassificationModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) @@ -74,9 +94,15 @@ final class DecisionTreeClassifier(override val uid: String) } val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) val strategy = getOldStrategy(categoricalFeatures, numClasses) - val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", - seed = 0L, parentUID = Some(uid)) - trees.head.asInstanceOf[DecisionTreeClassificationModel] + val model = getAlgorithm match { + case "byRow" => + val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, + featureSubsetStrategy = "all", seed = 0L, parentUID = Some(uid)) + trees.head + case "byCol" => + AltDT.train(oldDataset, strategy, parentUID = Some(uid)) + } + model.asInstanceOf[DecisionTreeClassificationModel] } /** (private[ml]) Create a Strategy instance to use with the old API. */ @@ -94,6 +120,8 @@ final class DecisionTreeClassifier(override val uid: String) object DecisionTreeClassifier { /** Accessor for supported impurities: entropy, gini */ final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities + + final val supportedAlgorithms: Array[String] = Array("byRow", "byCol") } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index a2bcd67401d0..8f07755878a9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.Experimental import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.tree.{DecisionTreeModel, DecisionTreeParams, Node, TreeRegressorParams} -import org.apache.spark.ml.tree.impl.RandomForest +import org.apache.spark.ml.tree.impl.{AltDT, RandomForest} import org.apache.spark.ml.util.{Identifiable, MetadataUtils} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint @@ -62,14 +62,39 @@ final class DecisionTreeRegressor(override val uid: String) override def setImpurity(value: String): this.type = super.setImpurity(value) + /** + * Algorithm used for learning. + * Supported: "byRow" or "byCol" (case sensitive). + * (default = "byRow") + * @group param + */ + val algorithm: Param[String] = new Param[String](this, "algorithm", "Algorithm used " + + "for learning. Supported options:" + + s" ${DecisionTreeRegressor.supportedAlgorithms.mkString(", ")}", + (value: String) => DecisionTreeRegressor.supportedAlgorithms.contains(value)) + + setDefault(algorithm -> "byRow") + + /** @group setParam */ + def setAlgorithm(value: String): this.type = set(algorithm, value) + + /** @group getParam */ + def getAlgorithm: String = $(algorithm) + override protected def train(dataset: DataFrame): DecisionTreeRegressionModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) val strategy = getOldStrategy(categoricalFeatures) - val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", - seed = 0L, parentUID = Some(uid)) - trees.head.asInstanceOf[DecisionTreeRegressionModel] + val model = getAlgorithm match { + case "byRow" => + val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, + featureSubsetStrategy = "all", seed = 0L, parentUID = Some(uid)) + trees.head + case "byCol" => + AltDT.train(oldDataset, strategy, parentUID = Some(uid)) + } + model.asInstanceOf[DecisionTreeRegressionModel] } /** (private[ml]) Create a Strategy instance to use with the old API. */ @@ -85,6 +110,8 @@ final class DecisionTreeRegressor(override val uid: String) object DecisionTreeRegressor { /** Accessor for supported impurities: variance */ final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities + + final val supportedAlgorithms: Array[String] = Array("byRow", "byCol") } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala deleted file mode 100644 index 1cdd058178a7..000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/AltDTModel.scala +++ /dev/null @@ -1,26 +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.ml.tree - -class AltDTModel { - -} - -class Prediction - -class AltDTNode(val prediction: Prediction, ) \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 44a6976cc3cd..5d867933332b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -20,7 +20,6 @@ package org.apache.spark.ml.tree.impl import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.TreeUtil._ @@ -51,14 +50,9 @@ import org.apache.spark.util.collection.BitSet * * TODO: Update to use a sparse column store. */ -@Experimental -class AltDT (private val strategy: Strategy) extends Serializable with Logging { +object AltDT extends Logging { - import AltDT.{FeatureVector, PartitionInfo, bitSubvectorFromSplit} - - strategy.assertValid() - - private def getImpurityAggregator(): ImpurityAggregatorSingle = { + private def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { strategy.impurity match { case Entropy => new EntropyAggregatorSingle(strategy.numClasses) case Gini => new GiniAggregatorSingle(strategy.numClasses) @@ -69,18 +63,21 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { /** * Method to train a decision tree model over an RDD. */ - def train(input: RDD[LabeledPoint], parentUID: Option[String] = None): DecisionTreeModel = { + def train( + input: RDD[LabeledPoint], + strategy: Strategy, + parentUID: Option[String] = None): DecisionTreeModel = { // TODO: Check validity of params - val rootNode = trainImpl(input) + val rootNode = trainImpl(input, strategy) RandomForest.finalizeTree(rootNode, strategy.algo, parentUID) } - def trainImpl(input: RDD[LabeledPoint]): Node = { + def trainImpl(input: RDD[LabeledPoint], strategy: Strategy): Node = { // The case with 1 node (depth = 0) is handled separately. // This allows all iterations in the depth > 0 case to use the same code. if (strategy.maxDepth == 0) { val impurityAggregator: ImpurityAggregatorSingle = - input.aggregate(getImpurityAggregator())( + input.aggregate(createImpurityAggregator(strategy))( (agg, lp) => agg.update(lp.label, 1.0), (agg1, agg2) => agg1.add(agg2)) val impurityCalculator = impurityAggregator.getCalculator @@ -148,11 +145,13 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { val localLabels = labelsBc.value // Iterate over the active nodes in the current level. activeNodes.iterator.map { nodeIndexInLevel: Int => + println(s"nodeIndexInLevel=$nodeIndexInLevel, nodeOffsets.length=${nodeOffsets.length}") val fromOffset = nodeOffsets(nodeIndexInLevel) val toOffset = nodeOffsets(nodeIndexInLevel + 1) val splitsAndStats = columns.map { col => - chooseSplit(col, localLabels, fromOffset, toOffset, getImpurityAggregator()) + chooseSplit(col, localLabels, fromOffset, toOffset, + createImpurityAggregator(strategy), strategy.minInfoGain) } // nodeIndexInLevel -> splitsAndStats.maxBy(_._2.gain) splitsAndStats.maxBy(_._2.gain) @@ -178,6 +177,7 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { val node = activeNodePeriphery(nodeIdx) node.predictionStats = new OldPredict(stats.prediction, -1) node.impurity = stats.impurity + println(s"nodeIdx: $nodeIdx, gain: ${stats.gain}") if (stats.gain > strategy.getMinInfoGain) { // TODO: Add prediction probability once that is added properly to trees node.leftChild = @@ -192,9 +192,11 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { Iterator() } } + println(s"activeNodePeriphery.length: ${activeNodePeriphery.length}") // We keep all old nodeOffsets and add one for each node split. // Each node split adds 2 nodes to activeNodePeriphery. numNodeOffsets = numNodeOffsets + activeNodePeriphery.length / 2 + println(s"numNodeOffsets: $numNodeOffsets") // TODO: Check to make sure we split something, and stop otherwise. val doneLearning = currentLevel >= strategy.maxDepth @@ -232,7 +234,7 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { // Broadcast aggregated bit vectors. On each partition, update instance--node map. val aggBitVectorsBc = input.sparkContext.broadcast(aggBitVectors) partitionInfos = partitionInfos.map { partitionInfo => - partitionInfo.update(aggBitVectorsBc.value, numNodeOffsets, activeNodePeriphery.length) + partitionInfo.update(aggBitVectorsBc.value, numNodeOffsets) } // TODO: unpersist aggBitVectorsBc after action. } @@ -261,7 +263,8 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { labels: Array[Double], fromOffset: Int, toOffset: Int, - impurityAgg: ImpurityAggregatorSingle): (Split, InfoGainStats) = { + impurityAgg: ImpurityAggregatorSingle, + minInfoGain: Double): (Split, InfoGainStats) = { val featureIndex = col.featureIndex val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) @@ -270,9 +273,9 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { labels.foreach(fullImpurityAgg.update(_, 1.0)) col.featureType match { case FeatureType.Categorical => - chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg) + chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg, minInfoGain) case FeatureType.Continuous => - chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg) + chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg, minInfoGain) } } @@ -281,7 +284,8 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { values: Seq[Double], labels: Seq[Double], leftImpurityAgg: ImpurityAggregatorSingle, - rightImpurityAgg: ImpurityAggregatorSingle): (Split, InfoGainStats) = ??? + rightImpurityAgg: ImpurityAggregatorSingle, + minInfoGain: Double): (Split, InfoGainStats) = ??? /** * Choose splitting rule: feature value <= threshold @@ -291,31 +295,38 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { values: Seq[Double], labels: Seq[Double], leftImpurityAgg: ImpurityAggregatorSingle, - rightImpurityAgg: ImpurityAggregatorSingle): (Split, InfoGainStats) = { + rightImpurityAgg: ImpurityAggregatorSingle, + minInfoGain: Double): (Split, InfoGainStats) = { val prediction = leftImpurityAgg.getCalculator.getPredict var bestThreshold: Double = Double.NegativeInfinity var bestLeftImpurityAgg = leftImpurityAgg.deepCopy() var bestGain: Double = 0.0 val fullImpurity = rightImpurityAgg.getCalculator.calculate() - var leftWeight: Double = 0.0 - var rightWeight: Double = rightImpurityAgg.getWeight + var leftCount: Double = 0.0 + var rightCount: Double = rightImpurityAgg.getCount + val fullCount: Double = rightCount + println(s"\nfeatureIndex: $featureIndex") values.zip(labels).foreach { case (value, label) => // Move this instance from right to left side of split. leftImpurityAgg.update(label, 1.0) rightImpurityAgg.update(label, -1.0) - leftWeight += 1.0 - rightWeight -= 1.0 + leftCount += 1.0 + rightCount -= 1.0 + val leftWeight = leftCount / fullCount + val rightWeight = rightCount / fullCount // Check gain val leftImpurity = leftImpurityAgg.getCalculator.calculate() val rightImpurity = rightImpurityAgg.getCalculator.calculate() val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity - if (gain > bestGain) { + print(s" gain=$gain ") + if (gain > bestGain && gain > minInfoGain) { bestThreshold = value leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) bestGain = gain } } + println() val leftImpurity = bestLeftImpurityAgg.getCalculator.calculate() val bestRightImpurityAgg = @@ -323,14 +334,10 @@ class AltDT (private val strategy: Strategy) extends Serializable with Logging { val rightImpurity = bestRightImpurityAgg.getCalculator.calculate() val bestGainStats = new InfoGainStats(prediction.predict, bestGain, fullImpurity, leftImpurity, rightImpurity, bestLeftImpurityAgg.getCalculator.getPredict, - bestRightImpurityAgg.getCalculator.getPredict) + bestRightImpurityAgg.getCalculator.getPredict) (new ContinuousSplit(featureIndex, bestThreshold), bestGainStats) } -} - -object AltDT extends Serializable with Logging { - /** * Feature vector types are based on (feature type, representation). * The feature type can be continuous or categorical. @@ -382,8 +389,8 @@ object AltDT extends Serializable with Logging { fromOffset: Int, toOffset: Int, split: Split): BitSubvector = { - val nodeRowIndices = col.indices.view.slice(fromOffset, toOffset) - val nodeRowValues = col.values.view.slice(fromOffset, toOffset) + val nodeRowIndices = col.indices.view.slice(fromOffset, toOffset).toArray + val nodeRowValues = col.values.view.slice(fromOffset, toOffset).toArray val nodeRowValuesSortedByIndices = nodeRowIndices.zip(nodeRowValues).sortBy(_._1).map(_._2) val bitv = new BitSubvector(fromOffset, toOffset) nodeRowValuesSortedByIndices.zipWithIndex.foreach { case (value, i) => @@ -435,7 +442,7 @@ object AltDT extends Serializable with Logging { * For instances at inactive (leaf) nodes, the value can be arbitrary. * @return Updated partition info */ - def update(bitVectors: Array[BitSubvector], newNumNodeOffsets: Int, numNewActiveNodes: Int): PartitionInfo = { + def update(bitVectors: Array[BitSubvector], newNumNodeOffsets: Int): PartitionInfo = { val newColumns = columns.map { oldCol => val col = oldCol.deepCopy() var curBitVecIdx = 0 @@ -446,8 +453,9 @@ object AltDT extends Serializable with Logging { val curBitVector = bitVectors(curBitVecIdx) // Sort range [from, to) based on indices. This is required to match the bit vector // across all workers. See [[bitSubvectorFromSplit]] for details. - val sortedRange = - col.indices.view.slice(from, to).zip(col.values.view.slice(from, to)).sortBy(_._1) + val rangeIndices = col.indices.view.slice(from, to).toArray + val rangeValues = col.values.view.slice(from, to).toArray + val sortedRange = rangeIndices.zip(rangeValues).sortBy(_._1) // Sort range [from, to) based on bit vector. sortedRange.zipWithIndex.map { case ((idx, value), i) => val bit = curBitVector.get(from + i) @@ -479,8 +487,12 @@ object AltDT extends Serializable with Logging { } } + assert(newNodeOffsets.map(_.length).sum == newNumNodeOffsets, + s"newNodeOffsets total size: ${newNodeOffsets.map(_.length).sum}," + + s" newNumNodeOffsets: $newNumNodeOffsets") + // Identify the new activeNodes based on the 2-level representation of the new nodeOffsets. - val newActiveNodes = new BitSet(newNumNodeOffsets) + val newActiveNodes = new BitSet(newNumNodeOffsets - 1) var newNodeOffsetsIdx = 0 newNodeOffsets.foreach { offsets => if (offsets.length == 2) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index d251fa43e77d..0edbef536e7b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.collection.BitSet -private class BitSubvector(val from: Int, val to: Int) extends Serializable { +private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializable { val numBits: Int = to - from @@ -43,7 +43,7 @@ private class BitSubvector(val from: Int, val to: Int) extends Serializable { } } -private object BitSubvector { +private[impl] object BitSubvector { def merge(parts1: Array[BitSubvector], parts2: Array[BitSubvector]): Array[BitSubvector] = { // Merge sorted parts1, parts2 diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 8b5128676d58..00da388fdf9e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -179,7 +179,7 @@ private[ml] object RandomForest extends Logging { } } - topNodes.map(finalizeTree(_, strategy.algo, parentUID)) + topNodes.map(lNode => finalizeTree(lNode.toNode, strategy.algo, parentUID)) } private[tree] def finalizeTree( diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala index 4d4b52b62939..512b4a19ea84 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala @@ -65,8 +65,8 @@ private[tree] abstract class ImpurityAggregatorSingle(val stats: Array[Double]) def deepCopy(): ImpurityAggregatorSingle - /** Total weight of instances in this aggregator */ - def getWeight: Double + /** Total (weighted) count of instances in this aggregator */ + def getCount: Double /** Resets this aggregator as though nothing has been added to it. */ def clear(): this.type = { @@ -100,7 +100,7 @@ private[tree] class EntropyAggregatorSingle private (stats: Array[Double]) override def deepCopy(): ImpurityAggregatorSingle = new EntropyAggregatorSingle(stats.clone()) - override def getWeight: Double = stats.sum + override def getCount: Double = stats.sum } /** @@ -124,7 +124,7 @@ private[tree] class GiniAggregatorSingle private (stats: Array[Double]) override def deepCopy(): ImpurityAggregatorSingle = new GiniAggregatorSingle(stats.clone()) - override def getWeight: Double = stats.sum + override def getCount: Double = stats.sum } /** @@ -148,5 +148,5 @@ private[tree] class VarianceAggregatorSingle tmp } - override def getWeight: Double = stats(0) + override def getCount: Double = stats(0) } From be5b2375417d84ab9591ec42d91711ba5bb7983a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 24 Jul 2015 14:57:05 -0700 Subject: [PATCH 09/34] debugging and adding unit tests --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 275 +++++++++++------- .../spark/ml/tree/impl/AltDTSuite.scala | 122 ++++++++ .../ml/tree/impl/BitSubvectorSuite.scala | 65 +++++ .../spark/ml/tree/impl/TreeUtilSuite.scala | 26 +- 4 files changed, 374 insertions(+), 114 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 5d867933332b..a36312d3f80f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -50,9 +50,9 @@ import org.apache.spark.util.collection.BitSet * * TODO: Update to use a sparse column store. */ -object AltDT extends Logging { +private[ml] object AltDT extends Logging { - private def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { + private[impl] def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { strategy.impurity match { case Entropy => new EntropyAggregatorSingle(strategy.numClasses) case Gini => new GiniAggregatorSingle(strategy.numClasses) @@ -72,7 +72,7 @@ object AltDT extends Logging { RandomForest.finalizeTree(rootNode, strategy.algo, parentUID) } - def trainImpl(input: RDD[LabeledPoint], strategy: Strategy): Node = { + private[impl] def trainImpl(input: RDD[LabeledPoint], strategy: Strategy): Node = { // The case with 1 node (depth = 0) is handled separately. // This allows all iterations in the depth > 0 case to use the same code. if (strategy.maxDepth == 0) { @@ -90,6 +90,7 @@ object AltDT extends Logging { // Or is the mapping implicit (i.e., not costly)? val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) val numRows: Int = colStoreInit.first()._2.size + println(s"numRows = $numRows") val labels = new Array[Double](numRows) input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => labels(rowIndex.toInt) = label @@ -113,10 +114,10 @@ object AltDT extends Logging { iterator.foreach(groupedCols += _) Iterator(groupedCols.toArray) } - groupedColStore.persist(StorageLevel.MEMORY_AND_DISK) + groupedColStore.repartition(1).persist(StorageLevel.MEMORY_AND_DISK) // TODO: remove repartition // Initialize partitions with 1 partition (i.e., each instance at the root node). - var partitionInfos: RDD[PartitionInfo] = groupedColStore.map { groupedCols => + var partitionInfosA: RDD[PartitionInfo] = groupedColStore.map { groupedCols => val initActive = new BitSet(1) initActive.setUntil(1) new PartitionInfo(groupedCols, Array[Int](0, numRows), initActive) @@ -124,74 +125,32 @@ object AltDT extends Logging { // Initialize model. // Note: We do not use node indices. - val rootNode = LearningNode.emptyNode(-1) + val rootNode = LearningNode.emptyNode(1) // TODO: remove node id // Active nodes (still being split), updated each iteration var activeNodePeriphery: Array[LearningNode] = Array(rootNode) var numNodeOffsets: Int = 2 + val partitionInfosDebug = new scala.collection.mutable.ArrayBuffer[RDD[PartitionInfo]]() + partitionInfosDebug.append(partitionInfosA) + // Iteratively learn, one level of the tree at a time. var currentLevel = 0 while (currentLevel < strategy.maxDepth) { + println(s"CURRENT LEVEL: $currentLevel") + Thread.sleep(10000) - // On each partition, for each feature on the partition, select the best split for each node. - // This will use: - // - groupedColStore (the features) - // - partitionInfos (the node -> instance mapping) - // - labelsBc (the labels column) - // Each worker returns: - // for each active node, best split + info gain - val partBestSplitsAndGains: RDD[Array[(Split, InfoGainStats)]] = partitionInfos.map { - case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => - val localLabels = labelsBc.value - // Iterate over the active nodes in the current level. - activeNodes.iterator.map { nodeIndexInLevel: Int => - println(s"nodeIndexInLevel=$nodeIndexInLevel, nodeOffsets.length=${nodeOffsets.length}") - val fromOffset = nodeOffsets(nodeIndexInLevel) - val toOffset = nodeOffsets(nodeIndexInLevel + 1) - val splitsAndStats = - columns.map { col => - chooseSplit(col, localLabels, fromOffset, toOffset, - createImpurityAggregator(strategy), strategy.minInfoGain) - } - // nodeIndexInLevel -> splitsAndStats.maxBy(_._2.gain) - splitsAndStats.maxBy(_._2.gain) - }.toArray - } + val partitionInfos = partitionInfosDebug.last + println(s"A: First partitionInfos' nodeOffsets: ${partitionInfos.first().nodeOffsets.mkString("(",",",")")}") - // Aggregate best split for each active node. + // Compute best split for each active node. val bestSplitsAndGains: Array[(Split, InfoGainStats)] = - partBestSplitsAndGains.reduce { case (splitsGains1, splitsGains2) => - splitsGains1.zip(splitsGains2).map { case ((split1, gain1), (split2, gain2)) => - if (gain1.gain >= gain2.gain) { - (split1, gain1) - } else { - (split2, gain2) - } - } - } + computeBestSplits(partitionInfos, labelsBc, strategy) + assert(activeNodePeriphery.length == bestSplitsAndGains.length) // Update current model and node periphery. // Note: This flatMap has side effects (on the model). - activeNodePeriphery = bestSplitsAndGains.zipWithIndex.flatMap { - case ((split, stats), nodeIdx) => - val node = activeNodePeriphery(nodeIdx) - node.predictionStats = new OldPredict(stats.prediction, -1) - node.impurity = stats.impurity - println(s"nodeIdx: $nodeIdx, gain: ${stats.gain}") - if (stats.gain > strategy.getMinInfoGain) { - // TODO: Add prediction probability once that is added properly to trees - node.leftChild = - Some(LearningNode(-1, stats.leftPredict, stats.leftImpurity, isLeaf = false)) - node.rightChild = - Some(LearningNode(-1, stats.rightPredict, stats.rightImpurity, isLeaf = false)) - node.split = Some(split) - node.stats = Some(stats.toOld) - Iterator(node.leftChild.get, node.rightChild.get) - } else { - node.isLeaf = true - Iterator() - } - } + activeNodePeriphery = + computeActiveNodePeriphery(activeNodePeriphery, bestSplitsAndGains, strategy.getMinInfoGain) println(s"activeNodePeriphery.length: ${activeNodePeriphery.length}") // We keep all old nodeOffsets and add one for each node split. // Each node split adds 2 nodes to activeNodePeriphery. @@ -199,43 +158,23 @@ object AltDT extends Logging { println(s"numNodeOffsets: $numNodeOffsets") // TODO: Check to make sure we split something, and stop otherwise. - val doneLearning = currentLevel >= strategy.maxDepth + val doneLearning = currentLevel + 1 >= strategy.maxDepth if (!doneLearning) { // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. - // - Send chosen splits to workers. - // - Each worker creates part of the bit vector corresponding to the splits it created. - // - Aggregate the partial bit vectors to create one vector (of length numRows). - // Correction: Aggregate only the pieces of that vector corresponding to instances at - // active nodes. - val bestSplitsBc: Broadcast[Array[Split]] = - input.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) - val workerBitSubvectors: RDD[Array[BitSubvector]] = partitionInfos.map { - case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => - val localBestSplits: Array[Split] = bestSplitsBc.value - // localFeatureIndex[feature index] = index into PartitionInfo.columns - val localFeatureIndex: Map[Int, Int] = columns.map(_.featureIndex).zipWithIndex.toMap - activeNodes.iterator.zip(localBestSplits.iterator).flatMap { - case (nodeIndexInLevel: Int, split: Split) => - if (localFeatureIndex.contains(split.featureIndex)) { - // This partition has the column (feature) used for this split. - val fromOffset = nodeOffsets(nodeIndexInLevel) - val toOffset = nodeOffsets(nodeIndexInLevel + 1) - val colIndex: Int = localFeatureIndex(split.featureIndex) - Iterator(bitSubvectorFromSplit(columns(colIndex), fromOffset, toOffset, split)) - } else { - Iterator() - } - }.toArray - } - val aggBitVectors: Array[BitSubvector] = workerBitSubvectors.reduce(BitSubvector.merge) - bestSplitsBc.unpersist() + val aggBitVectors: Array[BitSubvector] = collectBitVectors(partitionInfos, bestSplitsAndGains) + + println(s"B: First partitionInfos' nodeOffsets: ${partitionInfos.first().nodeOffsets.mkString("(",",",")")}") // Broadcast aggregated bit vectors. On each partition, update instance--node map. val aggBitVectorsBc = input.sparkContext.broadcast(aggBitVectors) - partitionInfos = partitionInfos.map { partitionInfo => + // partitionInfos = partitionInfos.map { partitionInfo => + val partitionInfosB = partitionInfos.map { partitionInfo => partitionInfo.update(aggBitVectorsBc.value, numNodeOffsets) } + partitionInfosDebug.append(partitionInfosB) + + println(s"C: First partitionInfos' nodeOffsets: ${partitionInfosB.first().nodeOffsets.mkString("(",",",")")}") // TODO: unpersist aggBitVectorsBc after action. } @@ -249,7 +188,125 @@ object AltDT extends Logging { rootNode.toNode } - // NOTE: The order of col does not correspond to the order of labels. Use the index in col. + /** + * Find the best splits for all active nodes. + * - On each partition, for each feature on the partition, select the best split for each node. + * Each worker returns: For each active node, best split + info gain + * - The splits across workers are aggregated to the driver. + * @param partitionInfos + * @param labelsBc + * @param strategy + * @return + */ + private[impl] def computeBestSplits( + partitionInfos: RDD[PartitionInfo], + labelsBc: Broadcast[Array[Double]], + strategy: Strategy): Array[(Split, InfoGainStats)] = { + // On each partition, for each feature on the partition, select the best split for each node. + // This will use: + // - groupedColStore (the features) + // - partitionInfos (the node -> instance mapping) + // - labelsBc (the labels column) + // Each worker returns: + // for each active node, best split + info gain + val partBestSplitsAndGains: RDD[Array[(Split, InfoGainStats)]] = partitionInfos.map { + case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => + val localLabels = labelsBc.value + // Iterate over the active nodes in the current level. + activeNodes.iterator.map { nodeIndexInLevel: Int => + println(s"nodeIndexInLevel=$nodeIndexInLevel, nodeOffsets.length=${nodeOffsets.length}") + val fromOffset = nodeOffsets(nodeIndexInLevel) + val toOffset = nodeOffsets(nodeIndexInLevel + 1) + val splitsAndStats = + columns.map { col => + chooseSplit(col, localLabels, fromOffset, toOffset, + createImpurityAggregator(strategy), strategy.minInfoGain) + } + // nodeIndexInLevel -> splitsAndStats.maxBy(_._2.gain) + splitsAndStats.maxBy(_._2.gain) + }.toArray + } + + // TODO: treeReduce + // Aggregate best split for each active node. + partBestSplitsAndGains.reduce { case (splitsGains1, splitsGains2) => + splitsGains1.zip(splitsGains2).map { case ((split1, gain1), (split2, gain2)) => + if (gain1.gain >= gain2.gain) { + (split1, gain1) + } else { + (split2, gain2) + } + } + } + } + + private[impl] def computeActiveNodePeriphery( + oldPeriphery: Array[LearningNode], + bestSplitsAndGains: Array[(Split, InfoGainStats)], + minInfoGain: Double): Array[LearningNode] = { + bestSplitsAndGains.zipWithIndex.flatMap { + case ((split, stats), nodeIdx) => + val node = oldPeriphery(nodeIdx) + node.predictionStats = new OldPredict(stats.prediction, -1) + node.impurity = stats.impurity + println(s"nodeIdx: $nodeIdx, gain: ${stats.gain}") + if (stats.gain > minInfoGain) { + // TODO: Add prediction probability once that is added properly to trees + node.leftChild = + Some(LearningNode(node.id * 2, stats.leftPredict, stats.leftImpurity, isLeaf = false)) // TODO: remove node id + node.rightChild = + Some(LearningNode(node.id * 2 + 1, stats.rightPredict, stats.rightImpurity, isLeaf = false)) // TODO: remove node id + node.split = Some(split) + node.stats = Some(stats.toOld) + println(s"DRIVER splitting node id=${node.id}: nodeIdx=$nodeIdx, gain=${stats.gain}") + Iterator(node.leftChild.get, node.rightChild.get) + } else { + node.isLeaf = true + Iterator() + } + } + } + + /** + * Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. + * - Send chosen splits to workers. + * - Each worker creates part of the bit vector corresponding to the splits it created. + * - Aggregate the partial bit vectors to create one vector (of length numRows). + * Correction: Aggregate only the pieces of that vector corresponding to instances at + * active nodes. + * @param partitionInfos RDD with feature data, plus current status metadata + * @param bestSplitsAndGains One (split, gain stats) pair per active node + * @return Array of bit vectors, ordered by offset ranges + */ + private[impl] def collectBitVectors( + partitionInfos: RDD[PartitionInfo], + bestSplitsAndGains: Array[(Split, InfoGainStats)]): Array[BitSubvector] = { + val bestSplitsBc: Broadcast[Array[Split]] = + partitionInfos.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) + val workerBitSubvectors: RDD[Array[BitSubvector]] = partitionInfos.map { + case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], + activeNodes: BitSet) => + val localBestSplits: Array[Split] = bestSplitsBc.value + // localFeatureIndex[feature index] = index into PartitionInfo.columns + val localFeatureIndex: Map[Int, Int] = columns.map(_.featureIndex).zipWithIndex.toMap + activeNodes.iterator.zip(localBestSplits.iterator).flatMap { + case (nodeIndexInLevel: Int, split: Split) => + if (localFeatureIndex.contains(split.featureIndex)) { + // This partition has the column (feature) used for this split. + val fromOffset = nodeOffsets(nodeIndexInLevel) + val toOffset = nodeOffsets(nodeIndexInLevel + 1) + val colIndex: Int = localFeatureIndex(split.featureIndex) + Iterator(bitSubvectorFromSplit(columns(colIndex), fromOffset, toOffset, split)) + } else { + Iterator() + } + }.toArray + } + val aggBitVectors: Array[BitSubvector] = workerBitSubvectors.reduce(BitSubvector.merge) + bestSplitsBc.unpersist() + aggBitVectors + } + /** * * @param col @@ -258,7 +315,7 @@ object AltDT extends Logging { * @param toOffset * @return */ - private def chooseSplit( + private[impl] def chooseSplit( col: FeatureVector, labels: Array[Double], fromOffset: Int, @@ -273,13 +330,15 @@ object AltDT extends Logging { labels.foreach(fullImpurityAgg.update(_, 1.0)) col.featureType match { case FeatureType.Categorical => - chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg, minInfoGain) + chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, + fullImpurityAgg, minInfoGain) case FeatureType.Continuous => - chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, fullImpurityAgg, minInfoGain) + chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, + fullImpurityAgg, minInfoGain) } } - private def chooseCategoricalSplit( + private[impl] def chooseCategoricalSplit( featureIndex: Int, values: Seq[Double], labels: Seq[Double], @@ -290,7 +349,7 @@ object AltDT extends Logging { /** * Choose splitting rule: feature value <= threshold */ - private def chooseContinuousSplit( + private[impl] def chooseContinuousSplit( featureIndex: Int, values: Seq[Double], labels: Seq[Double], @@ -347,7 +406,7 @@ object AltDT extends Logging { * TODO: Support sparse storage (to optimize deeper levels of the tree), and maybe compressed * storage (to optimize upper levels of the tree). */ - private class FeatureVector( + private[impl] class FeatureVector( val featureIndex: Int, val featureType: FeatureType, val values: Array[Double], @@ -356,9 +415,18 @@ object AltDT extends Logging { def deepCopy(): FeatureVector = new FeatureVector(featureIndex, featureType, values.clone(), indices.clone()) + + override def equals(other: Any): Boolean = { + other match { + case o: FeatureVector => + featureIndex == o.featureIndex && featureType == o.featureType && + values.sameElements(o.values) && indices.sameElements(o.indices) + case _ => false + } + } } - private object FeatureVector { + private[impl] object FeatureVector { /** Store column sorted by feature values. */ def fromOriginal( featureIndex: Int, @@ -384,7 +452,7 @@ object AltDT extends Logging { * second by sorted row indices within the node's rows. * bit[index in sorted array of row indices] = false for left, true for right */ - def bitSubvectorFromSplit( + private[impl] def bitSubvectorFromSplit( col: FeatureVector, fromOffset: Int, toOffset: Int, @@ -419,7 +487,7 @@ object AltDT extends Logging { * TODO: Should this (and even nodeOffsets) not be stored in PartitionInfo, * but instead on the driver? */ - private case class PartitionInfo( + private[impl] case class PartitionInfo( columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) @@ -470,6 +538,7 @@ object AltDT extends Logging { } // Create a 2-level representation of the new nodeOffsets (to be flattened). + println(s"initial nodeOffsets: ${nodeOffsets.mkString("(",",",")")}") val newNodeOffsets = nodeOffsets.map(Array(_)) var curBitVecIdx = 0 activeNodes.iterator.foreach { nodeIdx => @@ -480,13 +549,15 @@ object AltDT extends Logging { // Count number of values splitting to left vs. right val numRight = Range(from, to).count(curBitVector.get) val numLeft = to - from - numRight - if (numRight != 0 && numRight != 0) { + if (numLeft != 0 && numRight != 0) { // node is split - val oldOffset = newNodeOffsets(nodeIdx + 1).head - newNodeOffsets(nodeIdx + 1) = Array(oldOffset, oldOffset + numLeft) + val oldOffset = newNodeOffsets(nodeIdx).head + println(s"WORKER splitting node: nodeIdx=$nodeIdx, oldOffset=$oldOffset, numLeft=$numLeft") + newNodeOffsets(nodeIdx) = Array(oldOffset, oldOffset + numLeft) } } + println(s"newNodeOffsets: ${newNodeOffsets.map(_.mkString("(",",",")")).mkString("[", ", ", "]")}") assert(newNodeOffsets.map(_.length).sum == newNumNodeOffsets, s"newNodeOffsets total size: ${newNodeOffsets.map(_.length).sum}," + s" newNumNodeOffsets: $newNumNodeOffsets") diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala new file mode 100644 index 000000000000..7ce5078aa153 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala @@ -0,0 +1,122 @@ +/* + * 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.ml.tree.impl + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.tree.ContinuousSplit +import org.apache.spark.ml.tree.impl.AltDT.{FeatureVector, PartitionInfo} +import org.apache.spark.ml.tree.impl.TreeUtil._ +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.tree.configuration.FeatureType +import org.apache.spark.mllib.tree.model.Predict +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.util.collection.BitSet + +/** + * Test suite for [[AltDT]]. + */ +class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("FeatureVector") { + val v = new FeatureVector(1, FeatureType.Continuous, Array(0.1, 0.3, 0.7), Array(1, 2, 0)) + + val vCopy = v.deepCopy() + vCopy.values(0) = 1000 + assert(v.values(0) !== vCopy.values(0)) + + val original = Vectors.dense(0.7, 0.1, 0.3) + val v2 = FeatureVector.fromOriginal(1, FeatureType.Continuous, original) + assert(v === v2) + } + + test("PartitionInfo") { + val numRows = 4 + val col1 = + FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.8, 0.2, 0.1, 0.6)) + val col2 = + FeatureVector.fromOriginal(1, FeatureType.Categorical, Vectors.dense(0, 1, 0, 2)) + assert(col1.values.length === numRows) + assert(col2.values.length === numRows) + val nodeOffsets = Array(0, numRows) + val activeNodes = new BitSet(1) + activeNodes.set(0) + + val info = PartitionInfo(Array(col1, col2), nodeOffsets, activeNodes) + + // Create bitVector for splitting the 4 rows: L, R, L, R + // New groups are {0, 2}, {1, 3} + val bitVector = new BitSubvector(0, numRows) + bitVector.set(1) + bitVector.set(3) + + val newInfo = info.update(Array(bitVector), newNumNodeOffsets = 3) + + assert(newInfo.columns.length === 2) + val expectedCol1a = + new FeatureVector(0, FeatureType.Continuous, Array(0.1, 0.8, 0.2, 0.6), Array(2, 0, 1, 3)) + val expectedCol1b = + new FeatureVector(1, FeatureType.Categorical, Array(0, 0, 1, 2), Array(0, 2, 1, 3)) + assert(newInfo.columns(0) === expectedCol1a) + assert(newInfo.columns(1) === expectedCol1b) + assert(newInfo.nodeOffsets === Array(0, 2, 4)) + assert(newInfo.activeNodes.iterator.toSet === Set(0, 1)) + + // Create 2 bitVectors for splitting into: 0, 2, 1, 3 + val bv2a = new BitSubvector(0, 2) + bv2a.set(1) + val bv2b = new BitSubvector(2, 4) + bv2b.set(3) + + val newInfo2 = newInfo.update(Array(bv2a, bv2b), newNumNodeOffsets = 5) + + assert(newInfo2.columns.length === 2) + val expectedCol2a = + new FeatureVector(0, FeatureType.Continuous, Array(0.8, 0.1, 0.2, 0.6), Array(0, 2, 1, 3)) + val expectedCol2b = + new FeatureVector(1, FeatureType.Categorical, Array(0, 0, 1, 2), Array(0, 2, 1, 3)) + assert(newInfo2.columns(0) === expectedCol2a) + assert(newInfo2.columns(1) === expectedCol2b) + assert(newInfo2.nodeOffsets === Array(0, 1, 2, 3, 4)) + assert(newInfo2.activeNodes.iterator.toSet === Set(0, 1, 2, 3)) + } + + test("computeBestSplits") { + + } + + test("computeActiveNodePeriphery") { + } + + test("collectBitVectors") { + val col = + FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) + val numRows = col.values.length + val activeNodes = new BitSet(1) + activeNodes.set(0) + val info = PartitionInfo(Array(col), Array(0, numRows), activeNodes) + val partitionInfos = sc.parallelize(Seq(info)) + val bestSplitAndGain = (new ContinuousSplit(0, threshold = 0.5), + new InfoGainStats(0, 0, 0, 0, 0, new Predict(0, 0), new Predict(0, 0))) + val bitVectors = AltDT.collectBitVectors(partitionInfos, Array(bestSplitAndGain)) + assert(bitVectors.length === 1) + val bitv = bitVectors.head + assert(bitv.numBits === numRows) + assert(bitv.iterator.toArray === Array(3, 4)) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala new file mode 100644 index 000000000000..87775a1b6da8 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -0,0 +1,65 @@ +/* + * 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.ml.tree.impl + +import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.util.MLlibTestSparkContext + +/** + * Test suite for [[BitSubvector]]. + */ +class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("BitSubvector basic ops") { + val from = 1 + val to = 4 + val bs = new BitSubvector(from, to) + assert(bs.numBits === to - from) + Range(from, to).foreach(x => assert(!bs.get(x))) + val setVals = Array(from, to - 1) + setVals.foreach { x => + bs.set(x) + assert(bs.get(x)) + } + assert(bs.iterator.toSet === setVals.toSet) + } + + test("BitSubvector merge") { + val b1 = new BitSubvector(0, 5) + b1.set(1) + val b2 = new BitSubvector(5, 7) + b2.set(5) + val b3 = new BitSubvector(9, 12) + b3.set(11) + val parts1 = Array(b1) + val parts2 = Array(b2, b3) + val newParts = BitSubvector.merge(parts1, parts2) + + val r1 = new BitSubvector(0, 7) + r1.set(1) + r1.set(5) + val r2 = new BitSubvector(9, 12) + r2.set(11) + val expectedParts = Array(r1, r2) + newParts.zip(expectedParts).foreach { case (x, y) => + assert(x.from === y.from) + assert(x.to === x.to) + assert(x.iterator.toSet === y.iterator.toSet) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala index cd6e14f02b80..483fb8568f8b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeUtilSuite.scala @@ -19,28 +19,30 @@ package org.apache.spark.ml.tree.impl import scala.collection.mutable +import org.apache.spark.SparkFunSuite import org.apache.spark.ml.tree.impl.TreeUtil._ import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext /** * Test suite for [[TreeUtil]]. */ -class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { +class TreeUtilSuite extends SparkFunSuite with MLlibTestSparkContext { private def checkDense(rows: Seq[Vector]): Unit = { val numRowPartitions = 2 val rowStore = sc.parallelize(rows, numRowPartitions) val colStore = rowToColumnStoreDense(rowStore) - val numColPartitions = colStore.partitions.size + val numColPartitions = colStore.partitions.length val cols: Map[Int, Vector] = colStore.collect().toMap val numRows = rows.size if (numRows == 0) { - assert(cols.size == 0) + assert(cols.isEmpty) return } - val numCols = rows(0).size + val numCols = rows.head.size if (numCols == 0) { - assert(cols.size == 0) + assert(cols.isEmpty) return } rows.zipWithIndex.foreach { case (row, i) => @@ -50,7 +52,7 @@ class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { j += 1 } } - val expectedNumColPartitions = math.min(rowStore.partitions.size, numCols) + val expectedNumColPartitions = math.min(rowStore.partitions.length, numCols) assert(numColPartitions === expectedNumColPartitions) } @@ -59,17 +61,17 @@ class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { val overPartitionFactor = 2 val rowStore = sc.parallelize(rows, numRowPartitions) val colStore = rowToColumnStoreSparse(rowStore, overPartitionFactor) - val numColPartitions = colStore.partitions.size + val numColPartitions = colStore.partitions.length val cols: Map[Int, Vector] = colStore.collect().toMap val numRows = rows.size // Check cases with 0 rows or cols if (numRows == 0) { - assert(cols.size == 0) + assert(cols.isEmpty) return } - val numCols = rows(0).size + val numCols = rows.head.size if (numCols == 0) { - assert(cols.size == 0) + assert(cols.isEmpty) return } // Check values (and count non-zeros too) @@ -84,7 +86,7 @@ class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { } // Check sparsity val numNonZeros = cols.values.map { - case sv: SparseVector => sv.indices.size + case sv: SparseVector => sv.indices.length case _ => throw new RuntimeException( "checkSparse() found column which was not converted to SparseVector.") }.sum @@ -98,7 +100,7 @@ class TreeUtilSuite extends FunSuite with MLlibTestSparkContext { }.collect() colsByPartition.foreach { case (partitionIndex, partCols) => var j = 0 - while (j + 1 < partCols.size) { + while (j + 1 < partCols.length) { val curColIndex = partCols(j)._1 val nextColIndex = partCols(j + 1)._1 assert(curColIndex + 1 == nextColIndex) From 3c1a4d3c64b2a9dc940e8b7682d9cb0e0d04e7b5 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 26 Jul 2015 16:46:04 -0700 Subject: [PATCH 10/34] AltDT failing on imbalanced tree test. Going to refactor to broadcast active node set indices. --- .../examples/ml/DecisionTreeExample.scala | 4 +- .../org/apache/spark/ml/tree/impl/AltDT.scala | 140 +++++++++++++----- .../spark/ml/tree/impl/BitSubvector.scala | 48 +++--- .../spark/ml/tree/impl/AltDTSuite.scala | 92 +++++++++++- .../ml/tree/impl/BitSubvectorSuite.scala | 4 + 5 files changed, 224 insertions(+), 64 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index ca29c9a980d7..52616319ebd1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -286,14 +286,14 @@ object DecisionTreeExample { algo match { case "classification" => val treeModel = pipelineModel.stages.last.asInstanceOf[DecisionTreeClassificationModel] - if (treeModel.numNodes < 20) { + if (treeModel.numNodes < 200) { println(treeModel.toDebugString) // Print full model. } else { println(treeModel) // Print model summary. } case "regression" => val treeModel = pipelineModel.stages.last.asInstanceOf[DecisionTreeRegressionModel] - if (treeModel.numNodes < 20) { + if (treeModel.numNodes < 200) { println(treeModel.toDebugString) // Print full model. } else { println(treeModel) // Print model summary. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index a36312d3f80f..cd3862c3bc1d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -52,6 +52,9 @@ import org.apache.spark.util.collection.BitSet */ private[ml] object AltDT extends Logging { + private def myPrint(s: String = ""): Unit = print(s) + private def myPrintln(s: String = ""): Unit = println(s) + private[impl] def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { strategy.impurity match { case Entropy => new EntropyAggregatorSingle(strategy.numClasses) @@ -90,7 +93,7 @@ private[ml] object AltDT extends Logging { // Or is the mapping implicit (i.e., not costly)? val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) val numRows: Int = colStoreInit.first()._2.size - println(s"numRows = $numRows") + myPrintln(s"(D) numRows = $numRows") val labels = new Array[Double](numRows) input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => labels(rowIndex.toInt) = label @@ -108,15 +111,16 @@ private[ml] object AltDT extends Logging { FeatureVector.fromOriginal(featureIndex, FeatureType.Continuous, col) } } + myPrintln(s"colStore.numPartitions: ${colStore.partitions.length}") // Group columns together into one array of columns per partition. val groupedColStore: RDD[Array[FeatureVector]] = colStore.mapPartitions { iterator => val groupedCols = new ArrayBuffer[FeatureVector] iterator.foreach(groupedCols += _) - Iterator(groupedCols.toArray) + if (groupedCols.nonEmpty) Iterator(groupedCols.toArray) else Iterator() } groupedColStore.repartition(1).persist(StorageLevel.MEMORY_AND_DISK) // TODO: remove repartition - // Initialize partitions with 1 partition (i.e., each instance at the root node). + // Initialize partitions with 1 node (each instance at the root node). var partitionInfosA: RDD[PartitionInfo] = groupedColStore.map { groupedCols => val initActive = new BitSet(1) initActive.setUntil(1) @@ -135,36 +139,44 @@ private[ml] object AltDT extends Logging { // Iteratively learn, one level of the tree at a time. var currentLevel = 0 - while (currentLevel < strategy.maxDepth) { - println(s"CURRENT LEVEL: $currentLevel") - Thread.sleep(10000) + var doneLearning = false + while (currentLevel < strategy.maxDepth && !doneLearning) { + myPrintln("\n========================================\n") + myPrintln(s"(D) CURRENT LEVEL: $currentLevel") val partitionInfos = partitionInfosDebug.last - println(s"A: First partitionInfos' nodeOffsets: ${partitionInfos.first().nodeOffsets.mkString("(",",",")")}") + myPrintln(s"(D) A: Current partitionInfos:\n") + partitionInfos.collect().foreach(x => myPrintln(x.toString)) + myPrintln() // Compute best split for each active node. val bestSplitsAndGains: Array[(Split, InfoGainStats)] = computeBestSplits(partitionInfos, labelsBc, strategy) - assert(activeNodePeriphery.length == bestSplitsAndGains.length) + assert(activeNodePeriphery.length == bestSplitsAndGains.length, + s"activeNodePeriphery.length=${activeNodePeriphery.length} does not equal" + + s" bestSplitsAndGains.length=${bestSplitsAndGains.length}") // Update current model and node periphery. // Note: This flatMap has side effects (on the model). activeNodePeriphery = computeActiveNodePeriphery(activeNodePeriphery, bestSplitsAndGains, strategy.getMinInfoGain) - println(s"activeNodePeriphery.length: ${activeNodePeriphery.length}") + myPrintln(s"(D) activeNodePeriphery.length: ${activeNodePeriphery.length}") // We keep all old nodeOffsets and add one for each node split. // Each node split adds 2 nodes to activeNodePeriphery. numNodeOffsets = numNodeOffsets + activeNodePeriphery.length / 2 - println(s"numNodeOffsets: $numNodeOffsets") + myPrintln(s"(D) numNodeOffsets: $numNodeOffsets") + + // Filter active node periphery by impurity. + activeNodePeriphery = activeNodePeriphery.filter(_.impurity > 0.0) // TODO: Check to make sure we split something, and stop otherwise. - val doneLearning = currentLevel + 1 >= strategy.maxDepth + doneLearning = currentLevel + 1 >= strategy.maxDepth || activeNodePeriphery.length == 0 if (!doneLearning) { // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. val aggBitVectors: Array[BitSubvector] = collectBitVectors(partitionInfos, bestSplitsAndGains) - println(s"B: First partitionInfos' nodeOffsets: ${partitionInfos.first().nodeOffsets.mkString("(",",",")")}") + myPrintln(s"(D) B: First partitionInfos' nodeOffsets: ${partitionInfos.first().nodeOffsets.mkString("(",",",")")}") // Broadcast aggregated bit vectors. On each partition, update instance--node map. val aggBitVectorsBc = input.sparkContext.broadcast(aggBitVectors) @@ -172,9 +184,10 @@ private[ml] object AltDT extends Logging { val partitionInfosB = partitionInfos.map { partitionInfo => partitionInfo.update(aggBitVectorsBc.value, numNodeOffsets) } + partitionInfosB.cache().count() // TODO: remove partitionInfosDebug.append(partitionInfosB) - println(s"C: First partitionInfos' nodeOffsets: ${partitionInfosB.first().nodeOffsets.mkString("(",",",")")}") + myPrintln(s"(D) C: First partitionInfos' nodeOffsets: ${partitionInfosB.first().nodeOffsets.mkString("(",",",")")}") // TODO: unpersist aggBitVectorsBc after action. } @@ -211,10 +224,11 @@ private[ml] object AltDT extends Logging { // for each active node, best split + info gain val partBestSplitsAndGains: RDD[Array[(Split, InfoGainStats)]] = partitionInfos.map { case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => + myPrintln(s"(W) computeBestSplits(): activeNodes=${activeNodes.iterator.mkString(",")}") val localLabels = labelsBc.value // Iterate over the active nodes in the current level. activeNodes.iterator.map { nodeIndexInLevel: Int => - println(s"nodeIndexInLevel=$nodeIndexInLevel, nodeOffsets.length=${nodeOffsets.length}") + myPrintln(s"\t ~~> nodeIndexInLevel=$nodeIndexInLevel, nodeOffsets.length=${nodeOffsets.length}") val fromOffset = nodeOffsets(nodeIndexInLevel) val toOffset = nodeOffsets(nodeIndexInLevel + 1) val splitsAndStats = @@ -222,7 +236,7 @@ private[ml] object AltDT extends Logging { chooseSplit(col, localLabels, fromOffset, toOffset, createImpurityAggregator(strategy), strategy.minInfoGain) } - // nodeIndexInLevel -> splitsAndStats.maxBy(_._2.gain) + // We use Iterator and flatMap to handle empty partitions. splitsAndStats.maxBy(_._2.gain) }.toArray } @@ -240,6 +254,13 @@ private[ml] object AltDT extends Logging { } } + /** + * + * @param oldPeriphery + * @param bestSplitsAndGains + * @param minInfoGain + * @return + */ private[impl] def computeActiveNodePeriphery( oldPeriphery: Array[LearningNode], bestSplitsAndGains: Array[(Split, InfoGainStats)], @@ -249,7 +270,7 @@ private[ml] object AltDT extends Logging { val node = oldPeriphery(nodeIdx) node.predictionStats = new OldPredict(stats.prediction, -1) node.impurity = stats.impurity - println(s"nodeIdx: $nodeIdx, gain: ${stats.gain}") + myPrintln(s"(D) nodeIdx: $nodeIdx, gain: ${stats.gain}") if (stats.gain > minInfoGain) { // TODO: Add prediction probability once that is added properly to trees node.leftChild = @@ -258,7 +279,7 @@ private[ml] object AltDT extends Logging { Some(LearningNode(node.id * 2 + 1, stats.rightPredict, stats.rightImpurity, isLeaf = false)) // TODO: remove node id node.split = Some(split) node.stats = Some(stats.toOld) - println(s"DRIVER splitting node id=${node.id}: nodeIdx=$nodeIdx, gain=${stats.gain}") + myPrintln(s"(D) DRIVER splitting node id=${node.id}: nodeIdx=$nodeIdx, gain=${stats.gain}") Iterator(node.leftChild.get, node.rightChild.get) } else { node.isLeaf = true @@ -296,6 +317,7 @@ private[ml] object AltDT extends Logging { val fromOffset = nodeOffsets(nodeIndexInLevel) val toOffset = nodeOffsets(nodeIndexInLevel + 1) val colIndex: Int = localFeatureIndex(split.featureIndex) + myPrintln(s"(W) collectBitVectors(): fromOffset=$fromOffset, toOffset=$toOffset, colIndex=$colIndex, nodeIndexInLevel=$nodeIndexInLevel") Iterator(bitSubvectorFromSplit(columns(colIndex), fromOffset, toOffset, split)) } else { Iterator() @@ -325,9 +347,12 @@ private[ml] object AltDT extends Logging { val featureIndex = col.featureIndex val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) + myPrintln(s"(W) chooseSplit: feature=${col.featureIndex}, vals=${valuesForNode.mkString("(",",",")")}," + + s" labels=${labelsForNode.mkString("(",",",")")}," + + s" inds=${col.indices.view.slice(fromOffset, toOffset).mkString("(",",",")")}") impurityAgg.clear() val fullImpurityAgg = impurityAgg.deepCopy() - labels.foreach(fullImpurityAgg.update(_, 1.0)) + labelsForNode.foreach(fullImpurityAgg.update(_, 1.0)) col.featureType match { case FeatureType.Categorical => chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, @@ -365,27 +390,33 @@ private[ml] object AltDT extends Logging { var leftCount: Double = 0.0 var rightCount: Double = rightImpurityAgg.getCount val fullCount: Double = rightCount - println(s"\nfeatureIndex: $featureIndex") + myPrintln(s"(W) chooseContinuousSplit(): featureIndex=$featureIndex, values=${values.mkString(",")}") + var currentThreshold = values.headOption.getOrElse(bestThreshold) values.zip(labels).foreach { case (value, label) => + if (value != currentThreshold) { + // Check gain + val leftWeight = leftCount / fullCount + val rightWeight = rightCount / fullCount + val leftImpurity = leftImpurityAgg.getCalculator.calculate() + val rightImpurity = rightImpurityAgg.getCalculator.calculate() + val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity + myPrintln(s"\t --> gain=$gain, fullImpurity=$fullImpurity, leftWeight=$leftWeight," + + s" leftImpurity=$leftImpurity, rightWeight=$rightWeight, rightImpurity=$rightImpurity") + if (gain > bestGain && gain > minInfoGain) { + bestThreshold = currentThreshold + leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) + bestGain = gain + myPrintln(s"\t -> best update: bestThreshold=$bestThreshold, bestGain=$bestGain") + } + currentThreshold = value + } // Move this instance from right to left side of split. leftImpurityAgg.update(label, 1.0) rightImpurityAgg.update(label, -1.0) leftCount += 1.0 rightCount -= 1.0 - val leftWeight = leftCount / fullCount - val rightWeight = rightCount / fullCount - // Check gain - val leftImpurity = leftImpurityAgg.getCalculator.calculate() - val rightImpurity = rightImpurityAgg.getCalculator.calculate() - val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity - print(s" gain=$gain ") - if (gain > bestGain && gain > minInfoGain) { - bestThreshold = value - leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) - bestGain = gain - } } - println() + myPrintln() val leftImpurity = bestLeftImpurityAgg.getCalculator.calculate() val bestRightImpurityAgg = @@ -413,6 +444,16 @@ private[ml] object AltDT extends Logging { val indices: Array[Int]) extends Serializable { + /** For debugging */ + override def toString: String = { + " FeatureVector(" + + s" featureIndex: $featureIndex,\n" + + s" featureType: $featureType,\n" + + s" values: ${values.mkString(", ")},\n" + + s" indices: ${indices.mkString(", ")},\n" + + " )" + } + def deepCopy(): FeatureVector = new FeatureVector(featureIndex, featureType, values.clone(), indices.clone()) @@ -461,9 +502,17 @@ private[ml] object AltDT extends Logging { val nodeRowValues = col.values.view.slice(fromOffset, toOffset).toArray val nodeRowValuesSortedByIndices = nodeRowIndices.zip(nodeRowValues).sortBy(_._1).map(_._2) val bitv = new BitSubvector(fromOffset, toOffset) + myPrint(s"(W) bitSubvectorFromSplit(): nodeRowIndices=${nodeRowIndices.mkString("(",",",")")}," + + s" nodeRowValues=${nodeRowValues.mkString("(",",",")")}") nodeRowValuesSortedByIndices.zipWithIndex.foreach { case (value, i) => - if (!split.shouldGoLeft(value)) bitv.set(fromOffset + i) + if (!split.shouldGoLeft(value)) { + bitv.set(fromOffset + i) + myPrint("R ") + } else { + myPrint("L ") + } } + myPrintln() bitv } @@ -493,6 +542,17 @@ private[ml] object AltDT extends Logging { activeNodes: BitSet) extends Serializable { + /** For debugging */ + override def toString: String = { + "PartitionInfo(" + + " columns: {\n" + + columns.mkString(",\n") + + " },\n" + + s" nodeOffsets: ${nodeOffsets.mkString(", ")},\n" + + s" activeNodes: ${activeNodes.iterator.mkString(", ")},\n" + + ")\n" + } + /** * Update columns and nodeOffsets for the next level of the tree. * @@ -517,6 +577,7 @@ private[ml] object AltDT extends Logging { activeNodes.iterator.foreach { nodeIdx => val from = nodeOffsets(nodeIdx) val to = nodeOffsets(nodeIdx + 1) + // Note: Each node is guaranteed to be covered within 1 bit vector. if (bitVectors(curBitVecIdx).to <= from) curBitVecIdx += 1 val curBitVector = bitVectors(curBitVecIdx) // Sort range [from, to) based on indices. This is required to match the bit vector @@ -538,7 +599,7 @@ private[ml] object AltDT extends Logging { } // Create a 2-level representation of the new nodeOffsets (to be flattened). - println(s"initial nodeOffsets: ${nodeOffsets.mkString("(",",",")")}") + myPrintln(s"(W) initial nodeOffsets: ${nodeOffsets.mkString("(",",",")")}") val newNodeOffsets = nodeOffsets.map(Array(_)) var curBitVecIdx = 0 activeNodes.iterator.foreach { nodeIdx => @@ -546,20 +607,23 @@ private[ml] object AltDT extends Logging { val to = nodeOffsets(nodeIdx + 1) if (bitVectors(curBitVecIdx).to <= from) curBitVecIdx += 1 val curBitVector = bitVectors(curBitVecIdx) + assert(curBitVector.from <= from && to <= curBitVector.to) // Count number of values splitting to left vs. right val numRight = Range(from, to).count(curBitVector.get) val numLeft = to - from - numRight if (numLeft != 0 && numRight != 0) { // node is split val oldOffset = newNodeOffsets(nodeIdx).head - println(s"WORKER splitting node: nodeIdx=$nodeIdx, oldOffset=$oldOffset, numLeft=$numLeft") + myPrintln(s"(W) WORKER splitting node: nodeIdx=$nodeIdx, oldOffset=$oldOffset, numLeft=$numLeft, numRight=$numRight") newNodeOffsets(nodeIdx) = Array(oldOffset, oldOffset + numLeft) + } else { + myPrintln(s"(W) WORKER NOT splitting node: nodeIdx=$nodeIdx, oldOffset=${newNodeOffsets(nodeIdx).head}, numLeft=$numLeft, numRight=$numRight") } } - println(s"newNodeOffsets: ${newNodeOffsets.map(_.mkString("(",",",")")).mkString("[", ", ", "]")}") + myPrintln(s"(W) newNodeOffsets: ${newNodeOffsets.map(_.mkString("(",",",")")).mkString("[", ", ", "]")}") assert(newNodeOffsets.map(_.length).sum == newNumNodeOffsets, - s"newNodeOffsets total size: ${newNodeOffsets.map(_.length).sum}," + + s"(W) newNodeOffsets total size: ${newNodeOffsets.map(_.length).sum}," + s" newNumNodeOffsets: $newNumNodeOffsets") // Identify the new activeNodes based on the 2-level representation of the new nodeOffsets. @@ -575,7 +639,7 @@ private[ml] object AltDT extends Logging { } } - new PartitionInfo(newColumns, newNodeOffsets.flatten, newActiveNodes) + PartitionInfo(newColumns, newNodeOffsets.flatten, newActiveNodes) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index 0edbef536e7b..fe9b171cfd1b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -48,30 +48,34 @@ private[impl] object BitSubvector { def merge(parts1: Array[BitSubvector], parts2: Array[BitSubvector]): Array[BitSubvector] = { // Merge sorted parts1, parts2 val sortedSubvectors = (parts1 ++ parts2).sortBy(_.from) - // Merge adjacent PartialBitVectors (for adjacent node ranges) - val newSubvectorRanges: Array[(Int, Int)] = { - val newSubvRanges = ArrayBuffer.empty[(Int, Int)] - var i = 1 - var currentFrom = sortedSubvectors.head.from - while (i < sortedSubvectors.length) { - if (sortedSubvectors(i - 1).to != sortedSubvectors(i).from) { - newSubvRanges.append((currentFrom, sortedSubvectors(i - 1).to)) - currentFrom = sortedSubvectors(i).from + if (sortedSubvectors.nonEmpty) { + // Merge adjacent PartialBitVectors (for adjacent node ranges) + val newSubvectorRanges: Array[(Int, Int)] = { + val newSubvRanges = ArrayBuffer.empty[(Int, Int)] + var i = 1 + var currentFrom = sortedSubvectors.head.from + while (i < sortedSubvectors.length) { + if (sortedSubvectors(i - 1).to != sortedSubvectors(i).from) { + newSubvRanges.append((currentFrom, sortedSubvectors(i - 1).to)) + currentFrom = sortedSubvectors(i).from + } + i += 1 } - i += 1 + newSubvRanges.append((currentFrom, sortedSubvectors.last.to)) + newSubvRanges.toArray } - newSubvRanges.append((currentFrom, sortedSubvectors.last.to)) - newSubvRanges.toArray - } - val newSubvectors = newSubvectorRanges.map { case (from, to) => new BitSubvector(from, to) } - var curNewSubvIdx = 0 - sortedSubvectors.foreach { subv => - if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 - val newSubv = newSubvectors(curNewSubvIdx) - // TODO: More efficient (word-level) copy. - subv.iterator.foreach(idx => newSubv.set(idx)) + val newSubvectors = newSubvectorRanges.map { case (from, to) => new BitSubvector(from, to) } + var curNewSubvIdx = 0 + sortedSubvectors.foreach { subv => + if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 + val newSubv = newSubvectors(curNewSubvIdx) + // TODO: More efficient (word-level) copy. + subv.iterator.foreach(idx => newSubv.set(idx)) + } + assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check + newSubvectors + } else { + Array.empty[BitSubvector] } - assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check - newSubvectors } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala index 7ce5078aa153..cc232f909491 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.tree.ContinuousSplit +import org.apache.spark.ml.regression.DecisionTreeRegressor +import org.apache.spark.ml.tree.{LeafNode, InternalNode, ContinuousSplit} import org.apache.spark.ml.tree.impl.AltDT.{FeatureVector, PartitionInfo} import org.apache.spark.ml.tree.impl.TreeUtil._ import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.FeatureType import org.apache.spark.mllib.tree.model.Predict import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,6 +34,42 @@ import org.apache.spark.util.collection.BitSet */ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { + test("run deep example") { + val data = Range(0, 3).map(x => LabeledPoint(math.pow(x, 3), Vectors.dense(x))) + val df = sqlContext.createDataFrame(data) + val dt = new DecisionTreeRegressor() + .setFeaturesCol("features") // indexedFeatures + .setLabelCol("label") + .setMaxDepth(10) + .setAlgorithm("byCol") + val model = dt.fit(df) + println(model.toDebugString) // TODO: remove println + assert(model.rootNode.isInstanceOf[InternalNode]) + val root = model.rootNode.asInstanceOf[InternalNode] + assert(root.leftChild.isInstanceOf[LeafNode] && root.rightChild.isInstanceOf[InternalNode]) + val right = root.rightChild.asInstanceOf[InternalNode] + assert(right.leftChild.isInstanceOf[InternalNode], right.rightChild.isInstanceOf[InternalNode]) + } + + test("run example") { + val data = Range(0, 8).map(x => LabeledPoint(x, Vectors.dense(x))) + val df = sqlContext.createDataFrame(data) + val dt = new DecisionTreeRegressor() + .setFeaturesCol("features") // indexedFeatures + .setLabelCol("label") + .setMaxDepth(10) + .setAlgorithm("byCol") + val model = dt.fit(df) + println(model.toDebugString) // TODO: remove println + assert(model.rootNode.isInstanceOf[InternalNode]) + val root = model.rootNode.asInstanceOf[InternalNode] + assert(root.leftChild.isInstanceOf[InternalNode] && root.rightChild.isInstanceOf[InternalNode]) + val left = root.leftChild.asInstanceOf[InternalNode] + val right = root.rightChild.asInstanceOf[InternalNode] + val grandkids = Array(left.leftChild, left.rightChild, right.leftChild, right.rightChild) + assert(grandkids.forall(_.isInstanceOf[InternalNode])) + } + test("FeatureVector") { val v = new FeatureVector(1, FeatureType.Continuous, Array(0.1, 0.3, 0.7), Array(1, 2, 0)) @@ -95,6 +133,39 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { assert(newInfo2.activeNodes.iterator.toSet === Set(0, 1, 2, 3)) } + test("bitSubvectorFromSplit: 1 node") { + val col = + FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) + val fromOffset = 0 + val toOffset = col.values.length + val split = new ContinuousSplit(0, threshold = 0.5) + val bitv = AltDT.bitSubvectorFromSplit(col, fromOffset, toOffset, split) + assert(bitv.from === fromOffset) + assert(bitv.to === toOffset) + assert(bitv.iterator.toSet === Set(3, 4)) + } + + test("bitSubvectorFromSplit: 2 nodes") { + // Initially, 1 split: (0, 2, 4) | (1, 3) + val col = new FeatureVector(0, FeatureType.Continuous, Array(0.1, 0.2, 0.4, 0.6, 0.7), + Array(4, 2, 0, 1, 3)) + def checkSplit(fromOffset: Int, toOffset: Int, threshold: Double, expectedRight: Set[Int]): Unit = { + val split = new ContinuousSplit(0, threshold) + val bitv = AltDT.bitSubvectorFromSplit(col, fromOffset, toOffset, split) + assert(bitv.from === fromOffset) + assert(bitv.to === toOffset) + assert(bitv.iterator.toSet === expectedRight) + } + // Left child node + checkSplit(0, 3, 0.15, Set(0, 1)) + checkSplit(0, 3, 0.2, Set(0)) + checkSplit(0, 3, 0.5, Set()) + // Right child node + checkSplit(3, 5, 0.1, Set(3, 4)) + checkSplit(3, 5, 0.65, Set(4)) + checkSplit(3, 5, 0.8, Set()) + } + test("computeBestSplits") { } @@ -102,7 +173,7 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { test("computeActiveNodePeriphery") { } - test("collectBitVectors") { + test("collectBitVectors with 1 vector") { val col = FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) val numRows = col.values.length @@ -119,4 +190,21 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { assert(bitv.iterator.toArray === Array(3, 4)) } + test("collectBitVectors with 1 vector, with tied threshold") { + val col = new FeatureVector(0, FeatureType.Continuous, + Array(-4.0,-4.0,-2.0,-2.0,-1.0,-1.0,1.0,1.0), Array(3,7,2,6,1,5,0,4)) + val numRows = col.values.length + val activeNodes = new BitSet(1) + activeNodes.set(0) + val info = PartitionInfo(Array(col), Array(0, numRows), activeNodes) + val partitionInfos = sc.parallelize(Seq(info)) + val bestSplitAndGain = (new ContinuousSplit(0, threshold = -2.0), + new InfoGainStats(0, 0, 0, 0, 0, new Predict(0, 0), new Predict(0, 0))) + val bitVectors = AltDT.collectBitVectors(partitionInfos, Array(bestSplitAndGain)) + assert(bitVectors.length === 1) + val bitv = bitVectors.head + assert(bitv.numBits === numRows) + assert(bitv.iterator.toArray === Array(0, 1, 4, 5)) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index 87775a1b6da8..6e7b67623851 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -62,4 +62,8 @@ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(x.iterator.toSet === y.iterator.toSet) } } + + test("BitSubvector merge with empty BitSubvectors") { + val parts = BitSubvector.merge(Array.empty[BitSubvector], Array.empty[BitSubvector]) + } } From 15b009b37d59ea6f8c3e38f1a2994a6923403dec Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 27 Jul 2015 10:57:43 -0700 Subject: [PATCH 11/34] AltDT is working --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 28 +++++++++++-------- .../spark/ml/tree/impl/AltDTSuite.scala | 6 ++-- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index cd3862c3bc1d..9549cb168b19 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -52,8 +52,8 @@ import org.apache.spark.util.collection.BitSet */ private[ml] object AltDT extends Logging { - private def myPrint(s: String = ""): Unit = print(s) - private def myPrintln(s: String = ""): Unit = println(s) + private def myPrint(s: String = ""): Unit = return // print(s) + private def myPrintln(s: String = ""): Unit = return // println(s) private[impl] def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { strategy.impurity match { @@ -152,9 +152,13 @@ private[ml] object AltDT extends Logging { // Compute best split for each active node. val bestSplitsAndGains: Array[(Split, InfoGainStats)] = computeBestSplits(partitionInfos, labelsBc, strategy) + /* + // NOTE: The actual active nodes (activeNodePeriphery) may be a subset of the nodes under + // bestSplitsAndGains since assert(activeNodePeriphery.length == bestSplitsAndGains.length, s"activeNodePeriphery.length=${activeNodePeriphery.length} does not equal" + s" bestSplitsAndGains.length=${bestSplitsAndGains.length}") + */ // Update current model and node periphery. // Note: This flatMap has side effects (on the model). @@ -163,14 +167,15 @@ private[ml] object AltDT extends Logging { myPrintln(s"(D) activeNodePeriphery.length: ${activeNodePeriphery.length}") // We keep all old nodeOffsets and add one for each node split. // Each node split adds 2 nodes to activeNodePeriphery. + // TODO: Should this be calculated after filtering for impurity?? numNodeOffsets = numNodeOffsets + activeNodePeriphery.length / 2 myPrintln(s"(D) numNodeOffsets: $numNodeOffsets") // Filter active node periphery by impurity. - activeNodePeriphery = activeNodePeriphery.filter(_.impurity > 0.0) + val estimatedRemainingActive = activeNodePeriphery.count(_.impurity > 0.0) // TODO: Check to make sure we split something, and stop otherwise. - doneLearning = currentLevel + 1 >= strategy.maxDepth || activeNodePeriphery.length == 0 + doneLearning = currentLevel + 1 >= strategy.maxDepth || estimatedRemainingActive == 0 if (!doneLearning) { // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. @@ -255,11 +260,12 @@ private[ml] object AltDT extends Logging { } /** - * - * @param oldPeriphery - * @param bestSplitsAndGains - * @param minInfoGain - * @return + * On driver: Grow tree based on chosen splits, and compute new set of active nodes. + * @param oldPeriphery Old periphery of active nodes. + * @param bestSplitsAndGains Best (split, gain) pairs, which can be zipped with the old + * periphery. + * @param minInfoGain Threshold for min info gain required to split a node. + * @return New active node periphery */ private[impl] def computeActiveNodePeriphery( oldPeriphery: Array[LearningNode], @@ -268,11 +274,11 @@ private[ml] object AltDT extends Logging { bestSplitsAndGains.zipWithIndex.flatMap { case ((split, stats), nodeIdx) => val node = oldPeriphery(nodeIdx) - node.predictionStats = new OldPredict(stats.prediction, -1) - node.impurity = stats.impurity myPrintln(s"(D) nodeIdx: $nodeIdx, gain: ${stats.gain}") if (stats.gain > minInfoGain) { // TODO: Add prediction probability once that is added properly to trees + node.predictionStats = new OldPredict(stats.prediction, -1) + node.impurity = stats.impurity node.leftChild = Some(LearningNode(node.id * 2, stats.leftPredict, stats.leftImpurity, isLeaf = false)) // TODO: remove node id node.rightChild = diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala index cc232f909491..1972233f125b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala @@ -46,9 +46,9 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { println(model.toDebugString) // TODO: remove println assert(model.rootNode.isInstanceOf[InternalNode]) val root = model.rootNode.asInstanceOf[InternalNode] - assert(root.leftChild.isInstanceOf[LeafNode] && root.rightChild.isInstanceOf[InternalNode]) - val right = root.rightChild.asInstanceOf[InternalNode] - assert(right.leftChild.isInstanceOf[InternalNode], right.rightChild.isInstanceOf[InternalNode]) + assert(root.leftChild.isInstanceOf[InternalNode] && root.rightChild.isInstanceOf[LeafNode]) + val left = root.leftChild.asInstanceOf[InternalNode] + assert(left.leftChild.isInstanceOf[LeafNode], left.rightChild.isInstanceOf[LeafNode]) } test("run example") { From d58ce406972f19778ad45078107fbd3037f7e6a6 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 27 Jul 2015 11:24:59 -0700 Subject: [PATCH 12/34] removed debug printlns --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 39 ------------------- 1 file changed, 39 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 9549cb168b19..7e032ba5bd97 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -52,9 +52,6 @@ import org.apache.spark.util.collection.BitSet */ private[ml] object AltDT extends Logging { - private def myPrint(s: String = ""): Unit = return // print(s) - private def myPrintln(s: String = ""): Unit = return // println(s) - private[impl] def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { strategy.impurity match { case Entropy => new EntropyAggregatorSingle(strategy.numClasses) @@ -93,7 +90,6 @@ private[ml] object AltDT extends Logging { // Or is the mapping implicit (i.e., not costly)? val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) val numRows: Int = colStoreInit.first()._2.size - myPrintln(s"(D) numRows = $numRows") val labels = new Array[Double](numRows) input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => labels(rowIndex.toInt) = label @@ -111,7 +107,6 @@ private[ml] object AltDT extends Logging { FeatureVector.fromOriginal(featureIndex, FeatureType.Continuous, col) } } - myPrintln(s"colStore.numPartitions: ${colStore.partitions.length}") // Group columns together into one array of columns per partition. val groupedColStore: RDD[Array[FeatureVector]] = colStore.mapPartitions { iterator => val groupedCols = new ArrayBuffer[FeatureVector] @@ -141,13 +136,8 @@ private[ml] object AltDT extends Logging { var currentLevel = 0 var doneLearning = false while (currentLevel < strategy.maxDepth && !doneLearning) { - myPrintln("\n========================================\n") - myPrintln(s"(D) CURRENT LEVEL: $currentLevel") val partitionInfos = partitionInfosDebug.last - myPrintln(s"(D) A: Current partitionInfos:\n") - partitionInfos.collect().foreach(x => myPrintln(x.toString)) - myPrintln() // Compute best split for each active node. val bestSplitsAndGains: Array[(Split, InfoGainStats)] = @@ -164,12 +154,10 @@ private[ml] object AltDT extends Logging { // Note: This flatMap has side effects (on the model). activeNodePeriphery = computeActiveNodePeriphery(activeNodePeriphery, bestSplitsAndGains, strategy.getMinInfoGain) - myPrintln(s"(D) activeNodePeriphery.length: ${activeNodePeriphery.length}") // We keep all old nodeOffsets and add one for each node split. // Each node split adds 2 nodes to activeNodePeriphery. // TODO: Should this be calculated after filtering for impurity?? numNodeOffsets = numNodeOffsets + activeNodePeriphery.length / 2 - myPrintln(s"(D) numNodeOffsets: $numNodeOffsets") // Filter active node periphery by impurity. val estimatedRemainingActive = activeNodePeriphery.count(_.impurity > 0.0) @@ -181,8 +169,6 @@ private[ml] object AltDT extends Logging { // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. val aggBitVectors: Array[BitSubvector] = collectBitVectors(partitionInfos, bestSplitsAndGains) - myPrintln(s"(D) B: First partitionInfos' nodeOffsets: ${partitionInfos.first().nodeOffsets.mkString("(",",",")")}") - // Broadcast aggregated bit vectors. On each partition, update instance--node map. val aggBitVectorsBc = input.sparkContext.broadcast(aggBitVectors) // partitionInfos = partitionInfos.map { partitionInfo => @@ -192,7 +178,6 @@ private[ml] object AltDT extends Logging { partitionInfosB.cache().count() // TODO: remove partitionInfosDebug.append(partitionInfosB) - myPrintln(s"(D) C: First partitionInfos' nodeOffsets: ${partitionInfosB.first().nodeOffsets.mkString("(",",",")")}") // TODO: unpersist aggBitVectorsBc after action. } @@ -229,11 +214,9 @@ private[ml] object AltDT extends Logging { // for each active node, best split + info gain val partBestSplitsAndGains: RDD[Array[(Split, InfoGainStats)]] = partitionInfos.map { case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => - myPrintln(s"(W) computeBestSplits(): activeNodes=${activeNodes.iterator.mkString(",")}") val localLabels = labelsBc.value // Iterate over the active nodes in the current level. activeNodes.iterator.map { nodeIndexInLevel: Int => - myPrintln(s"\t ~~> nodeIndexInLevel=$nodeIndexInLevel, nodeOffsets.length=${nodeOffsets.length}") val fromOffset = nodeOffsets(nodeIndexInLevel) val toOffset = nodeOffsets(nodeIndexInLevel + 1) val splitsAndStats = @@ -274,7 +257,6 @@ private[ml] object AltDT extends Logging { bestSplitsAndGains.zipWithIndex.flatMap { case ((split, stats), nodeIdx) => val node = oldPeriphery(nodeIdx) - myPrintln(s"(D) nodeIdx: $nodeIdx, gain: ${stats.gain}") if (stats.gain > minInfoGain) { // TODO: Add prediction probability once that is added properly to trees node.predictionStats = new OldPredict(stats.prediction, -1) @@ -285,7 +267,6 @@ private[ml] object AltDT extends Logging { Some(LearningNode(node.id * 2 + 1, stats.rightPredict, stats.rightImpurity, isLeaf = false)) // TODO: remove node id node.split = Some(split) node.stats = Some(stats.toOld) - myPrintln(s"(D) DRIVER splitting node id=${node.id}: nodeIdx=$nodeIdx, gain=${stats.gain}") Iterator(node.leftChild.get, node.rightChild.get) } else { node.isLeaf = true @@ -323,7 +304,6 @@ private[ml] object AltDT extends Logging { val fromOffset = nodeOffsets(nodeIndexInLevel) val toOffset = nodeOffsets(nodeIndexInLevel + 1) val colIndex: Int = localFeatureIndex(split.featureIndex) - myPrintln(s"(W) collectBitVectors(): fromOffset=$fromOffset, toOffset=$toOffset, colIndex=$colIndex, nodeIndexInLevel=$nodeIndexInLevel") Iterator(bitSubvectorFromSplit(columns(colIndex), fromOffset, toOffset, split)) } else { Iterator() @@ -353,9 +333,6 @@ private[ml] object AltDT extends Logging { val featureIndex = col.featureIndex val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) - myPrintln(s"(W) chooseSplit: feature=${col.featureIndex}, vals=${valuesForNode.mkString("(",",",")")}," + - s" labels=${labelsForNode.mkString("(",",",")")}," + - s" inds=${col.indices.view.slice(fromOffset, toOffset).mkString("(",",",")")}") impurityAgg.clear() val fullImpurityAgg = impurityAgg.deepCopy() labelsForNode.foreach(fullImpurityAgg.update(_, 1.0)) @@ -396,7 +373,6 @@ private[ml] object AltDT extends Logging { var leftCount: Double = 0.0 var rightCount: Double = rightImpurityAgg.getCount val fullCount: Double = rightCount - myPrintln(s"(W) chooseContinuousSplit(): featureIndex=$featureIndex, values=${values.mkString(",")}") var currentThreshold = values.headOption.getOrElse(bestThreshold) values.zip(labels).foreach { case (value, label) => if (value != currentThreshold) { @@ -406,13 +382,10 @@ private[ml] object AltDT extends Logging { val leftImpurity = leftImpurityAgg.getCalculator.calculate() val rightImpurity = rightImpurityAgg.getCalculator.calculate() val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity - myPrintln(s"\t --> gain=$gain, fullImpurity=$fullImpurity, leftWeight=$leftWeight," + - s" leftImpurity=$leftImpurity, rightWeight=$rightWeight, rightImpurity=$rightImpurity") if (gain > bestGain && gain > minInfoGain) { bestThreshold = currentThreshold leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) bestGain = gain - myPrintln(s"\t -> best update: bestThreshold=$bestThreshold, bestGain=$bestGain") } currentThreshold = value } @@ -422,7 +395,6 @@ private[ml] object AltDT extends Logging { leftCount += 1.0 rightCount -= 1.0 } - myPrintln() val leftImpurity = bestLeftImpurityAgg.getCalculator.calculate() val bestRightImpurityAgg = @@ -508,17 +480,11 @@ private[ml] object AltDT extends Logging { val nodeRowValues = col.values.view.slice(fromOffset, toOffset).toArray val nodeRowValuesSortedByIndices = nodeRowIndices.zip(nodeRowValues).sortBy(_._1).map(_._2) val bitv = new BitSubvector(fromOffset, toOffset) - myPrint(s"(W) bitSubvectorFromSplit(): nodeRowIndices=${nodeRowIndices.mkString("(",",",")")}," + - s" nodeRowValues=${nodeRowValues.mkString("(",",",")")}") nodeRowValuesSortedByIndices.zipWithIndex.foreach { case (value, i) => if (!split.shouldGoLeft(value)) { bitv.set(fromOffset + i) - myPrint("R ") - } else { - myPrint("L ") } } - myPrintln() bitv } @@ -605,7 +571,6 @@ private[ml] object AltDT extends Logging { } // Create a 2-level representation of the new nodeOffsets (to be flattened). - myPrintln(s"(W) initial nodeOffsets: ${nodeOffsets.mkString("(",",",")")}") val newNodeOffsets = nodeOffsets.map(Array(_)) var curBitVecIdx = 0 activeNodes.iterator.foreach { nodeIdx => @@ -620,14 +585,10 @@ private[ml] object AltDT extends Logging { if (numLeft != 0 && numRight != 0) { // node is split val oldOffset = newNodeOffsets(nodeIdx).head - myPrintln(s"(W) WORKER splitting node: nodeIdx=$nodeIdx, oldOffset=$oldOffset, numLeft=$numLeft, numRight=$numRight") newNodeOffsets(nodeIdx) = Array(oldOffset, oldOffset + numLeft) - } else { - myPrintln(s"(W) WORKER NOT splitting node: nodeIdx=$nodeIdx, oldOffset=${newNodeOffsets(nodeIdx).head}, numLeft=$numLeft, numRight=$numRight") } } - myPrintln(s"(W) newNodeOffsets: ${newNodeOffsets.map(_.mkString("(",",",")")).mkString("[", ", ", "]")}") assert(newNodeOffsets.map(_.length).sum == newNumNodeOffsets, s"(W) newNodeOffsets total size: ${newNodeOffsets.map(_.length).sum}," + s" newNumNodeOffsets: $newNumNodeOffsets") From 40719aff3d7469e9ea803df45b6678363146ad07 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 25 Aug 2015 18:44:41 -0700 Subject: [PATCH 13/34] fix merge conflicts --- .../classification/DecisionTreeClassifier.scala | 1 - .../apache/spark/ml/tree/impl/RandomForest.scala | 15 ++++++--------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 0aab2cf257b0..41bd5a906dd4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -18,7 +18,6 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.tree.{DecisionTreeModel, DecisionTreeParams, Node, TreeClassifierParams} import org.apache.spark.ml.tree.impl.{AltDT, RandomForest} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 00da388fdf9e..525168fc476c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -179,27 +179,24 @@ private[ml] object RandomForest extends Logging { } } - topNodes.map(lNode => finalizeTree(lNode.toNode, strategy.algo, parentUID)) + topNodes.map(lNode => finalizeTree(lNode.toNode, strategy.algo, strategy.numClasses, parentUID)) } private[tree] def finalizeTree( rootNode: Node, algo: OldAlgo.Algo, + numClasses: Int, parentUID: Option[String]): DecisionTreeModel = { parentUID match { case Some(uid) => - if (strategy.algo == OldAlgo.Classification) { - topNodes.map { rootNode => - new DecisionTreeClassificationModel(uid, rootNode.toNode, strategy.getNumClasses) - } + if (algo == OldAlgo.Classification) { + new DecisionTreeClassificationModel(uid, rootNode, numClasses) } else { new DecisionTreeRegressionModel(uid, rootNode) } case None => - if (strategy.algo == OldAlgo.Classification) { - topNodes.map { rootNode => - new DecisionTreeClassificationModel(rootNode.toNode, strategy.getNumClasses) - } + if (algo == OldAlgo.Classification) { + new DecisionTreeClassificationModel(rootNode, numClasses) } else { new DecisionTreeRegressionModel(rootNode) } From 594157763ecfadcfb4b984bafc5b98176af5aa34 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 25 Aug 2015 18:58:47 -0700 Subject: [PATCH 14/34] modified AltDT to use ImpurityStats --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 7e032ba5bd97..a1e0a62f8f04 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy} -import org.apache.spark.mllib.tree.model.{Predict => OldPredict} +import org.apache.spark.mllib.tree.model.{Predict => OldPredict, ImpurityStats} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.BitSet @@ -69,7 +69,7 @@ private[ml] object AltDT extends Logging { parentUID: Option[String] = None): DecisionTreeModel = { // TODO: Check validity of params val rootNode = trainImpl(input, strategy) - RandomForest.finalizeTree(rootNode, strategy.algo, parentUID) + RandomForest.finalizeTree(rootNode, strategy.algo, strategy.numClasses, parentUID) } private[impl] def trainImpl(input: RDD[LabeledPoint], strategy: Strategy): Node = { @@ -81,7 +81,8 @@ private[ml] object AltDT extends Logging { (agg, lp) => agg.update(lp.label, 1.0), (agg1, agg2) => agg1.add(agg2)) val impurityCalculator = impurityAggregator.getCalculator - return new LeafNode(impurityCalculator.getPredict.predict, impurityCalculator.calculate()) + return new LeafNode(impurityCalculator.getPredict.predict, impurityCalculator.calculate(), + impurityCalculator) } // Prepare column store. @@ -329,7 +330,7 @@ private[ml] object AltDT extends Logging { fromOffset: Int, toOffset: Int, impurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, InfoGainStats) = { + minInfoGain: Double): (Split, ImpurityStats) = { val featureIndex = col.featureIndex val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) @@ -352,7 +353,7 @@ private[ml] object AltDT extends Logging { labels: Seq[Double], leftImpurityAgg: ImpurityAggregatorSingle, rightImpurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, InfoGainStats) = ??? + minInfoGain: Double): (Split, ImpurityStats) = ??? /** * Choose splitting rule: feature value <= threshold @@ -363,7 +364,7 @@ private[ml] object AltDT extends Logging { labels: Seq[Double], leftImpurityAgg: ImpurityAggregatorSingle, rightImpurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, InfoGainStats) = { + minInfoGain: Double): (Split, ImpurityStats) = { val prediction = leftImpurityAgg.getCalculator.getPredict var bestThreshold: Double = Double.NegativeInfinity @@ -397,13 +398,12 @@ private[ml] object AltDT extends Logging { } val leftImpurity = bestLeftImpurityAgg.getCalculator.calculate() - val bestRightImpurityAgg = - leftImpurityAgg.deepCopy().add(rightImpurityAgg).subtract(bestLeftImpurityAgg) + val fullImpurityAgg = leftImpurityAgg.deepCopy().add(rightImpurityAgg) + val bestRightImpurityAgg = fullImpurityAgg.deepCopy().subtract(bestLeftImpurityAgg) val rightImpurity = bestRightImpurityAgg.getCalculator.calculate() - val bestGainStats = new InfoGainStats(prediction.predict, bestGain, fullImpurity, - leftImpurity, rightImpurity, bestLeftImpurityAgg.getCalculator.getPredict, - bestRightImpurityAgg.getCalculator.getPredict) - (new ContinuousSplit(featureIndex, bestThreshold), bestGainStats) + val bestImpurityStats = new ImpurityStats(bestGain, fullImpurity, fullImpurityAgg.getCalculator, + bestLeftImpurityAgg.getCalculator, bestRightImpurityAgg.getCalculator) + (new ContinuousSplit(featureIndex, bestThreshold), bestImpurityStats) } /** From 248d0a7039bebcb6b972bd351bc7b4cb443c51c6 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 26 Aug 2015 11:41:04 -0700 Subject: [PATCH 15/34] fixed conflicts with master, and started to add support for categorical features --- .../scala/org/apache/spark/ml/tree/Node.scala | 2 +- .../org/apache/spark/ml/tree/impl/AltDT.scala | 104 ++++++++++++++---- .../tree/model/InformationGainStats.scala | 14 ++- 3 files changed, 93 insertions(+), 27 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index cd2493129390..4bc9cad1920a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -288,7 +288,7 @@ private[tree] object LearningNode { id: Int, isLeaf: Boolean, stats: ImpurityStats): LearningNode = { - new LearningNode(id, None, None, None, false, stats) + new LearningNode(id, None, None, None, isLeaf, stats) } /** Create an empty node with the given node index. Values must be set later on. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index a1e0a62f8f04..8120cca7e46b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -28,10 +28,10 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy} -import org.apache.spark.mllib.tree.model.{Predict => OldPredict, ImpurityStats} +import org.apache.spark.mllib.tree.model.ImpurityStats import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.collection.BitSet +import org.apache.spark.util.collection.{BitSet, OpenHashSet} /** @@ -85,6 +85,9 @@ private[ml] object AltDT extends Logging { impurityCalculator) } + // Choose categorical features to be ordered or unordered. + val unorderedFeatures: Set[Int] = chooseUnorderedFeatures(strategy) + // Prepare column store. // Note: rowToColumnStoreDense checks to make sure numRows < Int.MaxValue. // TODO: Is this mapping from arrays to iterators to arrays (when constructing learningData)? @@ -141,7 +144,7 @@ private[ml] object AltDT extends Logging { val partitionInfos = partitionInfosDebug.last // Compute best split for each active node. - val bestSplitsAndGains: Array[(Split, InfoGainStats)] = + val bestSplitsAndGains: Array[(Split, ImpurityStats)] = computeBestSplits(partitionInfos, labelsBc, strategy) /* // NOTE: The actual active nodes (activeNodePeriphery) may be a subset of the nodes under @@ -161,7 +164,7 @@ private[ml] object AltDT extends Logging { numNodeOffsets = numNodeOffsets + activeNodePeriphery.length / 2 // Filter active node periphery by impurity. - val estimatedRemainingActive = activeNodePeriphery.count(_.impurity > 0.0) + val estimatedRemainingActive = activeNodePeriphery.count(_.stats.impurity > 0.0) // TODO: Check to make sure we split something, and stop otherwise. doneLearning = currentLevel + 1 >= strategy.maxDepth || estimatedRemainingActive == 0 @@ -176,7 +179,7 @@ private[ml] object AltDT extends Logging { val partitionInfosB = partitionInfos.map { partitionInfo => partitionInfo.update(aggBitVectorsBc.value, numNodeOffsets) } - partitionInfosB.cache().count() // TODO: remove + partitionInfosB.cache().count() // TODO: remove. For some reason, this is needed to make things work. Probably messing up somewhere above... partitionInfosDebug.append(partitionInfosB) // TODO: unpersist aggBitVectorsBc after action. @@ -188,10 +191,69 @@ private[ml] object AltDT extends Logging { // Done with learning groupedColStore.unpersist() labelsBc.unpersist() - // TODO: return model rootNode.toNode } + /** + * Choose which categorical features are unordered vs. ordered. + * + * TODO: This was copied from DecisionTreeMetadata. When merging, make sure to eliminate duplicate code. + * @return Set of unordered feature indices + */ + private[impl] def chooseUnorderedFeatures(strategy: Strategy, numExamples: Long, numFeatures: Int): Set[Int] = { + val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt + if (maxPossibleBins < strategy.maxBins) { + logWarning(s"DecisionTree reducing maxBins from ${strategy.maxBins} to $maxPossibleBins" + + s" (= number of training instances)") + } + // We check the number of bins here against maxPossibleBins. + // This needs to be checked here instead of in Strategy since maxPossibleBins can be modified + // based on the number of training examples. + if (strategy.categoricalFeaturesInfo.nonEmpty) { + val maxCategoriesPerFeature = strategy.categoricalFeaturesInfo.values.max + val maxCategory = + strategy.categoricalFeaturesInfo.find(_._2 == maxCategoriesPerFeature).get._1 + require(maxCategoriesPerFeature <= maxPossibleBins, + s"DecisionTree requires maxBins (= $maxPossibleBins) to be at least as large as the " + + s"number of values in each categorical feature, but categorical feature $maxCategory " + + s"has $maxCategoriesPerFeature values. Considering remove this and other categorical " + + "features with a large number of values, or add more training examples.") + } + + val unorderedFeatures = new OpenHashSet[Int]() + val numBins = Array.fill[Int](numFeatures)(maxPossibleBins) + if (numClasses > 2) { + // Multiclass classification + val maxCategoriesForUnorderedFeature = + ((math.log(maxPossibleBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + // Hack: If a categorical feature has only 1 category, we treat it as continuous. + // TODO(SPARK-9957): Handle this properly by filtering out those features. + if (numCategories > 1) { + // Decide if some categorical features should be treated as unordered features, + // which require 2 * ((1 << numCategories - 1) - 1) bins. + // We do this check with log values to prevent overflows in case numCategories is large. + // The next check is equivalent to: 2 * ((1 << numCategories - 1) - 1) <= maxBins + if (numCategories <= maxCategoriesForUnorderedFeature) { + unorderedFeatures.add(featureIndex) + numBins(featureIndex) = numUnorderedBins(numCategories) + } else { + numBins(featureIndex) = numCategories + } + } + } + } else { + // Binary classification or regression + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + // If a categorical feature has only 1 category, we treat it as continuous: SPARK-9957 + if (numCategories > 1) { + numBins(featureIndex) = numCategories + } + } + } + + } + /** * Find the best splits for all active nodes. * - On each partition, for each feature on the partition, select the best split for each node. @@ -205,7 +267,7 @@ private[ml] object AltDT extends Logging { private[impl] def computeBestSplits( partitionInfos: RDD[PartitionInfo], labelsBc: Broadcast[Array[Double]], - strategy: Strategy): Array[(Split, InfoGainStats)] = { + strategy: Strategy): Array[(Split, ImpurityStats)] = { // On each partition, for each feature on the partition, select the best split for each node. // This will use: // - groupedColStore (the features) @@ -213,7 +275,7 @@ private[ml] object AltDT extends Logging { // - labelsBc (the labels column) // Each worker returns: // for each active node, best split + info gain - val partBestSplitsAndGains: RDD[Array[(Split, InfoGainStats)]] = partitionInfos.map { + val partBestSplitsAndGains: RDD[Array[(Split, ImpurityStats)]] = partitionInfos.map { case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => val localLabels = labelsBc.value // Iterate over the active nodes in the current level. @@ -253,21 +315,20 @@ private[ml] object AltDT extends Logging { */ private[impl] def computeActiveNodePeriphery( oldPeriphery: Array[LearningNode], - bestSplitsAndGains: Array[(Split, InfoGainStats)], + bestSplitsAndGains: Array[(Split, ImpurityStats)], minInfoGain: Double): Array[LearningNode] = { bestSplitsAndGains.zipWithIndex.flatMap { case ((split, stats), nodeIdx) => val node = oldPeriphery(nodeIdx) if (stats.gain > minInfoGain) { - // TODO: Add prediction probability once that is added properly to trees - node.predictionStats = new OldPredict(stats.prediction, -1) - node.impurity = stats.impurity - node.leftChild = - Some(LearningNode(node.id * 2, stats.leftPredict, stats.leftImpurity, isLeaf = false)) // TODO: remove node id - node.rightChild = - Some(LearningNode(node.id * 2 + 1, stats.rightPredict, stats.rightImpurity, isLeaf = false)) // TODO: remove node id + // TODO: remove node id + node.leftChild = Some(LearningNode(node.id * 2, isLeaf = false, + ImpurityStats(stats.leftImpurity, stats.leftImpurityCalculator))) + node.rightChild = Some(LearningNode(node.id * 2 + 1, isLeaf = false, + ImpurityStats(stats.rightImpurity, stats.rightImpurityCalculator))) node.split = Some(split) - node.stats = Some(stats.toOld) + node.isLeaf = false + node.stats = stats Iterator(node.leftChild.get, node.rightChild.get) } else { node.isLeaf = true @@ -289,7 +350,7 @@ private[ml] object AltDT extends Logging { */ private[impl] def collectBitVectors( partitionInfos: RDD[PartitionInfo], - bestSplitsAndGains: Array[(Split, InfoGainStats)]): Array[BitSubvector] = { + bestSplitsAndGains: Array[(Split, ImpurityStats)]): Array[BitSubvector] = { val bestSplitsBc: Broadcast[Array[Split]] = partitionInfos.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) val workerBitSubvectors: RDD[Array[BitSubvector]] = partitionInfos.map { @@ -331,7 +392,6 @@ private[ml] object AltDT extends Logging { toOffset: Int, impurityAgg: ImpurityAggregatorSingle, minInfoGain: Double): (Split, ImpurityStats) = { - val featureIndex = col.featureIndex val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) impurityAgg.clear() @@ -365,10 +425,8 @@ private[ml] object AltDT extends Logging { leftImpurityAgg: ImpurityAggregatorSingle, rightImpurityAgg: ImpurityAggregatorSingle, minInfoGain: Double): (Split, ImpurityStats) = { - val prediction = leftImpurityAgg.getCalculator.getPredict - var bestThreshold: Double = Double.NegativeInfinity - var bestLeftImpurityAgg = leftImpurityAgg.deepCopy() + val bestLeftImpurityAgg = leftImpurityAgg.deepCopy() var bestGain: Double = 0.0 val fullImpurity = rightImpurityAgg.getCalculator.calculate() var leftCount: Double = 0.0 @@ -397,10 +455,8 @@ private[ml] object AltDT extends Logging { rightCount -= 1.0 } - val leftImpurity = bestLeftImpurityAgg.getCalculator.calculate() val fullImpurityAgg = leftImpurityAgg.deepCopy().add(rightImpurityAgg) val bestRightImpurityAgg = fullImpurityAgg.deepCopy().subtract(bestLeftImpurityAgg) - val rightImpurity = bestRightImpurityAgg.getCalculator.calculate() val bestImpurityStats = new ImpurityStats(bestGain, fullImpurity, fullImpurityAgg.getCalculator, bestLeftImpurityAgg.getCalculator, bestRightImpurityAgg.getCalculator) (new ContinuousSplit(featureIndex, bestThreshold), bestImpurityStats) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 091a0462c204..edb4ad88c48e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -103,13 +103,13 @@ private[spark] class ImpurityStats( s"right impurity = $rightImpurity" } - def leftImpurity: Double = if (leftImpurityCalculator != null) { + lazy val leftImpurity: Double = if (leftImpurityCalculator != null) { leftImpurityCalculator.calculate() } else { -1.0 } - def rightImpurity: Double = if (rightImpurityCalculator != null) { + lazy val rightImpurity: Double = if (rightImpurityCalculator != null) { rightImpurityCalculator.calculate() } else { -1.0 @@ -118,6 +118,16 @@ private[spark] class ImpurityStats( private[spark] object ImpurityStats { + /** + * Create stats object missing the child node info. + */ + def apply( + impurity: Double, + impurityCalculator: ImpurityCalculator, + valid: Boolean = true): ImpurityStats = { + new ImpurityStats(Double.NaN, impurity, impurityCalculator, null, null, valid) + } + /** * Return an [[org.apache.spark.mllib.tree.model.ImpurityStats]] object to * denote that current split doesn't satisfies minimum info gain or From 0764b90d5b0a7b37031b36d8d9946e54d6d28a81 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 26 Aug 2015 18:30:33 -0700 Subject: [PATCH 16/34] Added computation of unordered features --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 32 ++++++++++++++----- .../tree/impl/DecisionTreeMetadata.scala | 2 +- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 8120cca7e46b..7df6c0a14ee7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -25,8 +25,7 @@ import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.TreeUtil._ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType -import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, FeatureType, Strategy} import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy} import org.apache.spark.mllib.tree.model.ImpurityStats import org.apache.spark.rdd.RDD @@ -73,6 +72,10 @@ private[ml] object AltDT extends Logging { } private[impl] def trainImpl(input: RDD[LabeledPoint], strategy: Strategy): Node = { + // Get data stats. + val numExamples: Long = input.count() + val numFeatures: Int = input.first().features.size + // The case with 1 node (depth = 0) is handled separately. // This allows all iterations in the depth > 0 case to use the same code. if (strategy.maxDepth == 0) { @@ -86,7 +89,7 @@ private[ml] object AltDT extends Logging { } // Choose categorical features to be ordered or unordered. - val unorderedFeatures: Set[Int] = chooseUnorderedFeatures(strategy) + val unorderedFeatures: Set[Int] = chooseUnorderedFeatures(strategy, numExamples, numFeatures) // Prepare column store. // Note: rowToColumnStoreDense checks to make sure numRows < Int.MaxValue. @@ -94,6 +97,7 @@ private[ml] object AltDT extends Logging { // Or is the mapping implicit (i.e., not costly)? val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) val numRows: Int = colStoreInit.first()._2.size + assert(numRows == numExamples.toInt, s"Data transpose had unknown failure") val labels = new Array[Double](numRows) input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => labels(rowIndex.toInt) = label @@ -200,7 +204,10 @@ private[ml] object AltDT extends Logging { * TODO: This was copied from DecisionTreeMetadata. When merging, make sure to eliminate duplicate code. * @return Set of unordered feature indices */ - private[impl] def chooseUnorderedFeatures(strategy: Strategy, numExamples: Long, numFeatures: Int): Set[Int] = { + private[impl] def chooseUnorderedFeatures( + strategy: Strategy, + numExamples: Long, + numFeatures: Int): Set[Int] = { val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt if (maxPossibleBins < strategy.maxBins) { logWarning(s"DecisionTree reducing maxBins from ${strategy.maxBins} to $maxPossibleBins" + @@ -222,7 +229,7 @@ private[ml] object AltDT extends Logging { val unorderedFeatures = new OpenHashSet[Int]() val numBins = Array.fill[Int](numFeatures)(maxPossibleBins) - if (numClasses > 2) { + if (strategy.algo == OldAlgo.Classification && strategy.numClasses > 2) { // Multiclass classification val maxCategoriesForUnorderedFeature = ((math.log(maxPossibleBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt @@ -251,9 +258,18 @@ private[ml] object AltDT extends Logging { } } } - + unorderedFeatures.iterator.toSet } + /** + * Given the arity of a categorical feature (arity = number of categories), + * return the number of bins for the feature if it is to be treated as an unordered feature. + * There is 1 split for every partitioning of categories into 2 disjoint, non-empty sets; + * there are math.pow(2, arity - 1) - 1 such splits. + * Each split has 2 corresponding bins. + */ + def numUnorderedBins(arity: Int): Int = 2 * ((1 << arity - 1) - 1) + /** * Find the best splits for all active nodes. * - On each partition, for each feature on the partition, select the best split for each node. @@ -473,7 +489,7 @@ private[ml] object AltDT extends Logging { */ private[impl] class FeatureVector( val featureIndex: Int, - val featureType: FeatureType, + val featureType: FeatureType.FeatureType, val values: Array[Double], val indices: Array[Int]) extends Serializable { @@ -505,7 +521,7 @@ private[ml] object AltDT extends Logging { /** Store column sorted by feature values. */ def fromOriginal( featureIndex: Int, - featureType: FeatureType, + featureType: FeatureType.FeatureType, featureVector: Vector): FeatureVector = { val (values, indices) = featureVector.toArray.zipWithIndex.sorted.unzip new FeatureVector(featureIndex, featureType, values.toArray, indices.toArray) 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 21ee49c45788..375bea0d56ef 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 @@ -205,7 +205,7 @@ private[spark] object DecisionTreeMetadata extends Logging { buildMetadata(input, strategy, numTrees = 1, featureSubsetStrategy = "all") } - /** + /** * Given the arity of a categorical feature (arity = number of categories), * return the number of bins for the feature if it is to be treated as an unordered feature. * There is 1 split for every partitioning of categories into 2 disjoint, non-empty sets; From ceaf5f560dca98a87b303d1cc41f9e46a877bbe7 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 27 Aug 2015 12:41:56 -0700 Subject: [PATCH 17/34] refactored to prepare for adding categorical feature support --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 190 ++++++++---------- 1 file changed, 85 insertions(+), 105 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 7df6c0a14ee7..eaea1e18f2cf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -26,7 +26,7 @@ import org.apache.spark.ml.tree.impl.TreeUtil._ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, FeatureType, Strategy} -import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy} +import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy, Impurity} import org.apache.spark.mllib.tree.model.ImpurityStats import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -51,12 +51,45 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet} */ private[ml] object AltDT extends Logging { - private[impl] def createImpurityAggregator(strategy: Strategy): ImpurityAggregatorSingle = { - strategy.impurity match { - case Entropy => new EntropyAggregatorSingle(strategy.numClasses) - case Gini => new GiniAggregatorSingle(strategy.numClasses) - case Variance => new VarianceAggregatorSingle + private[impl] class AltDTMetadata( + val numClasses: Int, + val maxBins: Int, + val minInfoGain: Double, + val impurity: Impurity) extends Serializable { + + private val maxCategoriesForUnorderedFeature = + ((math.log(maxBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt + + def isClassification: Boolean = numClasses >= 2 + + /** + * Indicates whether a categorical feature should be treated as unordered. + * + * TODO(SPARK-9957): If a categorical feature has only 1 category, we treat it as continuous. + * Later, handle this properly by filtering out those features. + */ + def isUnorderedFeature(numCategories: Int): Boolean = { + + // Decide if some categorical features should be treated as unordered features, + // which require 2 * ((1 << numCategories - 1) - 1) bins. + // We do this check with log values to prevent overflows in case numCategories is large. + // The last inequality is equivalent to: 2 * ((1 << numCategories - 1) - 1) <= maxBins + numClasses > 2 && numCategories > 1 && + numCategories <= maxCategoriesForUnorderedFeature } + + def createImpurityAggregator(): ImpurityAggregatorSingle = { + impurity match { + case Entropy => new EntropyAggregatorSingle(numClasses) + case Gini => new GiniAggregatorSingle(numClasses) + case Variance => new VarianceAggregatorSingle + } + } + } + + private[impl] object AltDTMetadata { + def fromStrategy(strategy: Strategy) = new AltDTMetadata(strategy.numClasses, strategy.maxBins, + strategy.minInfoGain, strategy.impurity) } /** @@ -72,15 +105,13 @@ private[ml] object AltDT extends Logging { } private[impl] def trainImpl(input: RDD[LabeledPoint], strategy: Strategy): Node = { - // Get data stats. - val numExamples: Long = input.count() - val numFeatures: Int = input.first().features.size + val metadata = AltDTMetadata.fromStrategy(strategy) // The case with 1 node (depth = 0) is handled separately. // This allows all iterations in the depth > 0 case to use the same code. if (strategy.maxDepth == 0) { val impurityAggregator: ImpurityAggregatorSingle = - input.aggregate(createImpurityAggregator(strategy))( + input.aggregate(metadata.createImpurityAggregator())( (agg, lp) => agg.update(lp.label, 1.0), (agg1, agg2) => agg1.add(agg2)) val impurityCalculator = impurityAggregator.getCalculator @@ -88,16 +119,12 @@ private[ml] object AltDT extends Logging { impurityCalculator) } - // Choose categorical features to be ordered or unordered. - val unorderedFeatures: Set[Int] = chooseUnorderedFeatures(strategy, numExamples, numFeatures) - // Prepare column store. // Note: rowToColumnStoreDense checks to make sure numRows < Int.MaxValue. // TODO: Is this mapping from arrays to iterators to arrays (when constructing learningData)? // Or is the mapping implicit (i.e., not costly)? val colStoreInit: RDD[(Int, Vector)] = rowToColumnStoreDense(input.map(_.features)) val numRows: Int = colStoreInit.first()._2.size - assert(numRows == numExamples.toInt, s"Data transpose had unknown failure") val labels = new Array[Double](numRows) input.map(_.label).zipWithIndex().collect().foreach { case (label: Double, rowIndex: Long) => labels(rowIndex.toInt) = label @@ -107,15 +134,15 @@ private[ml] object AltDT extends Logging { // rather than 1 copy per worker. This means a lot of random accesses. // We could improve this by applying first-level sorting (by node) to labels. + // TODO: RIGHT HERE NOW: JUST ADDED ISUNORDERED + // Sort each column by feature values. val colStore: RDD[FeatureVector] = colStoreInit.map { case (featureIndex: Int, col: Vector) => - if (strategy.categoricalFeaturesInfo.contains(featureIndex)) { - FeatureVector.fromOriginal(featureIndex, FeatureType.Categorical, col) - } else { - FeatureVector.fromOriginal(featureIndex, FeatureType.Continuous, col) - } + val featureArity: Int = strategy.categoricalFeaturesInfo.getOrElse(featureIndex, 0) + FeatureVector.fromOriginal(featureIndex, featureArity, col) } // Group columns together into one array of columns per partition. + // TODO: Test avoiding this grouping, and see if it matters. val groupedColStore: RDD[Array[FeatureVector]] = colStore.mapPartitions { iterator => val groupedCols = new ArrayBuffer[FeatureVector] iterator.foreach(groupedCols += _) @@ -126,7 +153,7 @@ private[ml] object AltDT extends Logging { // Initialize partitions with 1 node (each instance at the root node). var partitionInfosA: RDD[PartitionInfo] = groupedColStore.map { groupedCols => val initActive = new BitSet(1) - initActive.setUntil(1) + initActive.set(0) new PartitionInfo(groupedCols, Array[Int](0, numRows), initActive) } @@ -149,7 +176,7 @@ private[ml] object AltDT extends Logging { // Compute best split for each active node. val bestSplitsAndGains: Array[(Split, ImpurityStats)] = - computeBestSplits(partitionInfos, labelsBc, strategy) + computeBestSplits(partitionInfos, labelsBc, metadata) /* // NOTE: The actual active nodes (activeNodePeriphery) may be a subset of the nodes under // bestSplitsAndGains since @@ -198,69 +225,6 @@ private[ml] object AltDT extends Logging { rootNode.toNode } - /** - * Choose which categorical features are unordered vs. ordered. - * - * TODO: This was copied from DecisionTreeMetadata. When merging, make sure to eliminate duplicate code. - * @return Set of unordered feature indices - */ - private[impl] def chooseUnorderedFeatures( - strategy: Strategy, - numExamples: Long, - numFeatures: Int): Set[Int] = { - val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt - if (maxPossibleBins < strategy.maxBins) { - logWarning(s"DecisionTree reducing maxBins from ${strategy.maxBins} to $maxPossibleBins" + - s" (= number of training instances)") - } - // We check the number of bins here against maxPossibleBins. - // This needs to be checked here instead of in Strategy since maxPossibleBins can be modified - // based on the number of training examples. - if (strategy.categoricalFeaturesInfo.nonEmpty) { - val maxCategoriesPerFeature = strategy.categoricalFeaturesInfo.values.max - val maxCategory = - strategy.categoricalFeaturesInfo.find(_._2 == maxCategoriesPerFeature).get._1 - require(maxCategoriesPerFeature <= maxPossibleBins, - s"DecisionTree requires maxBins (= $maxPossibleBins) to be at least as large as the " + - s"number of values in each categorical feature, but categorical feature $maxCategory " + - s"has $maxCategoriesPerFeature values. Considering remove this and other categorical " + - "features with a large number of values, or add more training examples.") - } - - val unorderedFeatures = new OpenHashSet[Int]() - val numBins = Array.fill[Int](numFeatures)(maxPossibleBins) - if (strategy.algo == OldAlgo.Classification && strategy.numClasses > 2) { - // Multiclass classification - val maxCategoriesForUnorderedFeature = - ((math.log(maxPossibleBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt - strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => - // Hack: If a categorical feature has only 1 category, we treat it as continuous. - // TODO(SPARK-9957): Handle this properly by filtering out those features. - if (numCategories > 1) { - // Decide if some categorical features should be treated as unordered features, - // which require 2 * ((1 << numCategories - 1) - 1) bins. - // We do this check with log values to prevent overflows in case numCategories is large. - // The next check is equivalent to: 2 * ((1 << numCategories - 1) - 1) <= maxBins - if (numCategories <= maxCategoriesForUnorderedFeature) { - unorderedFeatures.add(featureIndex) - numBins(featureIndex) = numUnorderedBins(numCategories) - } else { - numBins(featureIndex) = numCategories - } - } - } - } else { - // Binary classification or regression - strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => - // If a categorical feature has only 1 category, we treat it as continuous: SPARK-9957 - if (numCategories > 1) { - numBins(featureIndex) = numCategories - } - } - } - unorderedFeatures.iterator.toSet - } - /** * Given the arity of a categorical feature (arity = number of categories), * return the number of bins for the feature if it is to be treated as an unordered feature. @@ -277,13 +241,13 @@ private[ml] object AltDT extends Logging { * - The splits across workers are aggregated to the driver. * @param partitionInfos * @param labelsBc - * @param strategy + * @param metadata * @return */ private[impl] def computeBestSplits( partitionInfos: RDD[PartitionInfo], labelsBc: Broadcast[Array[Double]], - strategy: Strategy): Array[(Split, ImpurityStats)] = { + metadata: AltDTMetadata): Array[(Split, ImpurityStats)] = { // On each partition, for each feature on the partition, select the best split for each node. // This will use: // - groupedColStore (the features) @@ -300,8 +264,7 @@ private[ml] object AltDT extends Logging { val toOffset = nodeOffsets(nodeIndexInLevel + 1) val splitsAndStats = columns.map { col => - chooseSplit(col, localLabels, fromOffset, toOffset, - createImpurityAggregator(strategy), strategy.minInfoGain) + chooseSplit(col, localLabels, fromOffset, toOffset, metadata) } // We use Iterator and flatMap to handle empty partitions. splitsAndStats.maxBy(_._2.gain) @@ -406,24 +369,36 @@ private[ml] object AltDT extends Logging { labels: Array[Double], fromOffset: Int, toOffset: Int, - impurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, ImpurityStats) = { + metadata: AltDTMetadata): (Split, ImpurityStats) = { + val impurityAgg = metadata.createImpurityAggregator() val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) impurityAgg.clear() val fullImpurityAgg = impurityAgg.deepCopy() labelsForNode.foreach(fullImpurityAgg.update(_, 1.0)) - col.featureType match { - case FeatureType.Categorical => - chooseCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, - fullImpurityAgg, minInfoGain) - case FeatureType.Continuous => - chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, - fullImpurityAgg, minInfoGain) + if (col.isCategorical) { + if (metadata.isUnorderedFeature(col.featureArity)) { + chooseUnorderedCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, + fullImpurityAgg, metadata.minInfoGain) + } else { + chooseOrderedCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, + fullImpurityAgg, metadata.minInfoGain) + } + } else { + chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, + fullImpurityAgg, metadata.minInfoGain) } } - private[impl] def chooseCategoricalSplit( + private[impl] def chooseOrderedCategoricalSplit( + featureIndex: Int, + values: Seq[Double], + labels: Seq[Double], + leftImpurityAgg: ImpurityAggregatorSingle, + rightImpurityAgg: ImpurityAggregatorSingle, + minInfoGain: Double): (Split, ImpurityStats) = ??? + + private[impl] def chooseUnorderedCategoricalSplit( featureIndex: Int, values: Seq[Double], labels: Seq[Double], @@ -486,32 +461,37 @@ private[ml] object AltDT extends Logging { * These values are currently stored in a dense representation only. * TODO: Support sparse storage (to optimize deeper levels of the tree), and maybe compressed * storage (to optimize upper levels of the tree). + * @param featureArity For categorical features, this gives the number of categories. + * For continuous features, this should be set to 0. */ private[impl] class FeatureVector( val featureIndex: Int, - val featureType: FeatureType.FeatureType, + val featureArity: Int, val values: Array[Double], val indices: Array[Int]) extends Serializable { + def isCategorical: Boolean = featureArity > 0 + /** For debugging */ override def toString: String = { " FeatureVector(" + s" featureIndex: $featureIndex,\n" + - s" featureType: $featureType,\n" + + s" featureType: ${if (featureArity == 0) "Continuous" else "Categorical"},\n" + + s" featureArity: $featureArity,\n" + s" values: ${values.mkString(", ")},\n" + s" indices: ${indices.mkString(", ")},\n" + " )" } def deepCopy(): FeatureVector = - new FeatureVector(featureIndex, featureType, values.clone(), indices.clone()) + new FeatureVector(featureIndex, featureArity, values.clone(), indices.clone()) override def equals(other: Any): Boolean = { other match { case o: FeatureVector => - featureIndex == o.featureIndex && featureType == o.featureType && - values.sameElements(o.values) && indices.sameElements(o.indices) + featureIndex == o.featureIndex && featureArity == o.featureArity && + values.sameElements(o.values) && indices.sameElements(o.indices) case _ => false } } @@ -521,10 +501,10 @@ private[ml] object AltDT extends Logging { /** Store column sorted by feature values. */ def fromOriginal( featureIndex: Int, - featureType: FeatureType.FeatureType, + featureArity: Int, featureVector: Vector): FeatureVector = { val (values, indices) = featureVector.toArray.zipWithIndex.sorted.unzip - new FeatureVector(featureIndex, featureType, values.toArray, indices.toArray) + new FeatureVector(featureIndex, featureArity, values.toArray, indices.toArray) } } From 675951e4ffd24fe0da1218e72210ceabfa9f7472 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 16 Sep 2015 10:13:48 -0700 Subject: [PATCH 18/34] Added support for ordered categorical features, but not tested yet --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 164 +++++++++++++++--- .../org/apache/spark/ml/tree/impurities.scala | 8 +- 2 files changed, 149 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index eaea1e18f2cf..98fb1a797e22 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -62,6 +62,8 @@ private[ml] object AltDT extends Logging { def isClassification: Boolean = numClasses >= 2 + def isMulticlass: Boolean = numClasses > 2 + /** * Indicates whether a categorical feature should be treated as unordered. * @@ -69,12 +71,11 @@ private[ml] object AltDT extends Logging { * Later, handle this properly by filtering out those features. */ def isUnorderedFeature(numCategories: Int): Boolean = { - // Decide if some categorical features should be treated as unordered features, // which require 2 * ((1 << numCategories - 1) - 1) bins. // We do this check with log values to prevent overflows in case numCategories is large. // The last inequality is equivalent to: 2 * ((1 << numCategories - 1) - 1) <= maxBins - numClasses > 2 && numCategories > 1 && + isMulticlass && numCategories > 1 && numCategories <= maxCategoriesForUnorderedFeature } @@ -109,6 +110,8 @@ private[ml] object AltDT extends Logging { // The case with 1 node (depth = 0) is handled separately. // This allows all iterations in the depth > 0 case to use the same code. + // TODO: Check that learning works when maxDepth > 0 but learning stops at 1 node (because of + // other parameters). if (strategy.maxDepth == 0) { val impurityAggregator: ImpurityAggregatorSingle = input.aggregate(metadata.createImpurityAggregator())( @@ -370,41 +373,154 @@ private[ml] object AltDT extends Logging { fromOffset: Int, toOffset: Int, metadata: AltDTMetadata): (Split, ImpurityStats) = { - val impurityAgg = metadata.createImpurityAggregator() val valuesForNode = col.values.view.slice(fromOffset, toOffset) val labelsForNode = col.indices.view.slice(fromOffset, toOffset).map(labels.apply) - impurityAgg.clear() - val fullImpurityAgg = impurityAgg.deepCopy() - labelsForNode.foreach(fullImpurityAgg.update(_, 1.0)) if (col.isCategorical) { if (metadata.isUnorderedFeature(col.featureArity)) { - chooseUnorderedCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, - fullImpurityAgg, metadata.minInfoGain) + chooseUnorderedCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, metadata, + col.featureArity) } else { - chooseOrderedCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, - fullImpurityAgg, metadata.minInfoGain) + chooseOrderedCategoricalSplit(col.featureIndex, valuesForNode, labelsForNode, metadata, + col.featureArity) } } else { - chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, impurityAgg, - fullImpurityAgg, metadata.minInfoGain) + chooseContinuousSplit(col.featureIndex, valuesForNode, labelsForNode, metadata) } } + /** + * Find the best split for an ordered categorical feature at a single node. + * + * Algorithm: + * - For each category, compute a "centroid." + * - For multiclass classification, the centroid is the label impurity. + * - For binary classification and regression, the centroid is the average label. + * - Sort the centroids, and consider splits anywhere in this order. + * Thus, with K categories, we consider K - 1 possible splits. + * + * @param featureIndex Index of feature being split. + * @param values Feature values at this node. Sorted in increasing order. + * @param labels Labels corresponding to values, in the same order. + * @return (best split, corresponding impurity statistics) + */ private[impl] def chooseOrderedCategoricalSplit( featureIndex: Int, values: Seq[Double], labels: Seq[Double], - leftImpurityAgg: ImpurityAggregatorSingle, - rightImpurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, ImpurityStats) = ??? + metadata: AltDTMetadata, + featureArity: Int): (Split, ImpurityStats) = { + // TODO: Support high-arity features by using a single array to hold the stats. + + // aggStats(category) = label statistics for category + val aggStats = Array.tabulate[ImpurityAggregatorSingle](featureArity)( + _ => metadata.createImpurityAggregator()) + values.zip(labels).foreach { case (cat, label) => + aggStats(cat.toInt).update(label) + } + + // Compute centroids. centroidsForCategories is a list: (category, centroid) + val centroidsForCategories: Seq[(Int, Double)] = if (metadata.isMulticlass) { + // For categorical variables in multiclass classification, + // the bins are ordered by the impurity of their corresponding labels. + Range(0, featureArity).map { case featureValue => + val categoryStats = aggStats(featureValue) + val centroid = if (categoryStats.getCount != 0) { + categoryStats.getCalculator.calculate() + } else { + Double.MaxValue + } + (featureValue, centroid) + } + } else if (metadata.isClassification) { // binary classification + // For categorical variables in binary classification, + // the bins are ordered by the centroid of their corresponding labels. + Range(0, featureArity).map { case featureValue => + val categoryStats = aggStats(featureValue) + val centroid = if (categoryStats.getCount != 0) { + assert(categoryStats.stats.length == 2) + (categoryStats.stats(1) - categoryStats.stats(0)) / categoryStats.getCount + } else { + Double.MaxValue + } + (featureValue, centroid) + } + } else { // regression + // For categorical variables in regression, + // the bins are ordered by the centroid of their corresponding labels. + Range(0, featureArity).map { case featureValue => + val categoryStats = aggStats(featureValue) + val centroid = if (categoryStats.getCount != 0) { + categoryStats.getCalculator.predict + } else { + Double.MaxValue + } + (featureValue, centroid) + } + } + + // TODO: RIGHT HERE NOW + + logDebug("Centroids for categorical variable: " + centroidsForCategories.mkString(",")) + + val categoriesSortedByCentroid: List[Int] = centroidsForCategories.toList.sortBy(_._2).map(_._1) + + // Cumulative sums of bin statistics for left, right parts of split. + val leftImpurityAgg = metadata.createImpurityAggregator() + val rightImpurityAgg = metadata.createImpurityAggregator() + aggStats.foreach(rightImpurityAgg.add) + + var bestSplitIndex: Int = -1 // index into categoriesSortedByCentroid + val bestLeftImpurityAgg = leftImpurityAgg.deepCopy() + var bestGain: Double = -1.0 + val fullImpurity = rightImpurityAgg.getCalculator.calculate() + var leftCount: Double = 0.0 + var rightCount: Double = rightImpurityAgg.getCount + val fullCount: Double = rightCount + + val numSplits = categoriesSortedByCentroid.length - 1 + var sortedCatIndex = 0 + while (sortedCatIndex < numSplits) { + val cat = categoriesSortedByCentroid(sortedCatIndex) + // Update left, right stats + val catStats = aggStats(cat) + leftImpurityAgg.add(catStats) + rightImpurityAgg.subtract(catStats) + leftCount += catStats.getCount + rightCount -= catStats.getCount + // Compute impurity + val leftWeight = leftCount / fullCount + val rightWeight = rightCount / fullCount + val leftImpurity = leftImpurityAgg.getCalculator.calculate() + val rightImpurity = rightImpurityAgg.getCalculator.calculate() + val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity + if (gain > bestGain && gain > metadata.minInfoGain) { + bestSplitIndex = sortedCatIndex + leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) + bestGain = gain + } + sortedCatIndex += 1 + } + + assert(bestSplitIndex != -1, "Unknown error in AltDT split selection for ordered categorical" + + s" variable with numSplits = $numSplits.") + + val categoriesForSplit = + categoriesSortedByCentroid.slice(0, bestSplitIndex + 1).map(_.toDouble) + val bestFeatureSplit = + new CategoricalSplit(featureIndex, categoriesForSplit.toArray, featureArity) + val fullImpurityAgg = leftImpurityAgg.deepCopy().add(rightImpurityAgg) + val bestRightImpurityAgg = fullImpurityAgg.deepCopy().subtract(bestLeftImpurityAgg) + val bestImpurityStats = new ImpurityStats(bestGain, fullImpurity, fullImpurityAgg.getCalculator, + bestLeftImpurityAgg.getCalculator, bestRightImpurityAgg.getCalculator) + (bestFeatureSplit, bestImpurityStats) + } private[impl] def chooseUnorderedCategoricalSplit( featureIndex: Int, values: Seq[Double], labels: Seq[Double], - leftImpurityAgg: ImpurityAggregatorSingle, - rightImpurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, ImpurityStats) = ??? + metadata: AltDTMetadata, + featureArity: Int): (Split, ImpurityStats) = ??? /** * Choose splitting rule: feature value <= threshold @@ -413,9 +529,12 @@ private[ml] object AltDT extends Logging { featureIndex: Int, values: Seq[Double], labels: Seq[Double], - leftImpurityAgg: ImpurityAggregatorSingle, - rightImpurityAgg: ImpurityAggregatorSingle, - minInfoGain: Double): (Split, ImpurityStats) = { + metadata: AltDTMetadata): (Split, ImpurityStats) = { + + val leftImpurityAgg = metadata.createImpurityAggregator() + val rightImpurityAgg = metadata.createImpurityAggregator() + labels.foreach(rightImpurityAgg.update(_, 1.0)) + var bestThreshold: Double = Double.NegativeInfinity val bestLeftImpurityAgg = leftImpurityAgg.deepCopy() var bestGain: Double = 0.0 @@ -432,7 +551,7 @@ private[ml] object AltDT extends Logging { val leftImpurity = leftImpurityAgg.getCalculator.calculate() val rightImpurity = rightImpurityAgg.getCalculator.calculate() val gain = fullImpurity - leftWeight * leftImpurity - rightWeight * rightImpurity - if (gain > bestGain && gain > minInfoGain) { + if (gain > bestGain && gain > metadata.minInfoGain) { bestThreshold = currentThreshold leftImpurityAgg.stats.copyToArray(bestLeftImpurityAgg.stats) bestGain = gain @@ -613,6 +732,7 @@ private[ml] object AltDT extends Logging { sortedRange.zipWithIndex.map { case ((idx, value), i) => val bit = curBitVector.get(from + i) // TODO: In-place merge, rather than general sort. + // TODO: We don't actually need to sort the categorical features using our approach. (bit, value, idx) }.sorted.zipWithIndex.foreach { case ((bit, value, idx), i) => col.values(from + i) = value diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala index 512b4a19ea84..88dc9c2d4c25 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impurities.scala @@ -55,11 +55,17 @@ private[tree] abstract class ImpurityAggregatorSingle(val stats: Array[Double]) } /** - * Update stats with the given label. + * Update stats with the given label and instance weight. * @return This aggregator (modified). */ def update(label: Double, instanceWeight: Double): this.type + /** + * Update stats with the given label. + * @return This aggregator (modified). + */ + def update(label: Double): this.type = update(label, 1.0) + /** Get an [[ImpurityCalculator]] for the current stats. */ def getCalculator: ImpurityCalculator From 1855176fa571adc6eec59ac26c64430a135563e7 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 23 Sep 2015 22:18:43 -0700 Subject: [PATCH 19/34] Added a few unit tests --- .../org/apache/spark/ml/tree/impl/AltDT.scala | 16 +-- .../spark/ml/tree/impl/AltDTSuite.scala | 109 +++++++++++++----- 2 files changed, 90 insertions(+), 35 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala index 98fb1a797e22..ff4eea2ef336 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/AltDT.scala @@ -205,7 +205,8 @@ private[ml] object AltDT extends Logging { if (!doneLearning) { // Aggregate bit vector (1 bit/instance) indicating whether each instance goes left/right. - val aggBitVectors: Array[BitSubvector] = collectBitVectors(partitionInfos, bestSplitsAndGains) + val aggBitVectors: Array[BitSubvector] = + collectBitVectors(partitionInfos, bestSplitsAndGains.map(_._1)) // Broadcast aggregated bit vectors. On each partition, update instance--node map. val aggBitVectorsBc = input.sparkContext.broadcast(aggBitVectors) @@ -327,14 +328,13 @@ private[ml] object AltDT extends Logging { * Correction: Aggregate only the pieces of that vector corresponding to instances at * active nodes. * @param partitionInfos RDD with feature data, plus current status metadata - * @param bestSplitsAndGains One (split, gain stats) pair per active node + * @param bestSplits Split for each active node * @return Array of bit vectors, ordered by offset ranges */ private[impl] def collectBitVectors( partitionInfos: RDD[PartitionInfo], - bestSplitsAndGains: Array[(Split, ImpurityStats)]): Array[BitSubvector] = { - val bestSplitsBc: Broadcast[Array[Split]] = - partitionInfos.sparkContext.broadcast(bestSplitsAndGains.map(_._1)) + bestSplits: Array[Split]): Array[BitSubvector] = { + val bestSplitsBc: Broadcast[Array[Split]] = partitionInfos.sparkContext.broadcast(bestSplits) val workerBitSubvectors: RDD[Array[BitSubvector]] = partitionInfos.map { case PartitionInfo(columns: Array[FeatureVector], nodeOffsets: Array[Int], activeNodes: BitSet) => @@ -360,6 +360,10 @@ private[ml] object AltDT extends Logging { } /** + * Choose the best split for a feature at a node. + * + * TODO: Return null or None when the split is invalid, such as putting all instances on one + * child node. * * @param col * @param labels @@ -458,8 +462,6 @@ private[ml] object AltDT extends Logging { } } - // TODO: RIGHT HERE NOW - logDebug("Centroids for categorical variable: " + centroidsForCategories.mkString(",")) val categoriesSortedByCentroid: List[Int] = centroidsForCategories.toList.sortBy(_._2).map(_._1) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala index 1972233f125b..6f05640a08a1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/AltDTSuite.scala @@ -20,12 +20,11 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite import org.apache.spark.ml.regression.DecisionTreeRegressor import org.apache.spark.ml.tree.{LeafNode, InternalNode, ContinuousSplit} -import org.apache.spark.ml.tree.impl.AltDT.{FeatureVector, PartitionInfo} +import org.apache.spark.ml.tree.impl.AltDT.{AltDTMetadata, FeatureVector, PartitionInfo} import org.apache.spark.ml.tree.impl.TreeUtil._ import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.FeatureType -import org.apache.spark.mllib.tree.model.Predict +import org.apache.spark.mllib.tree.impurity.{Variance, Gini, Entropy, Impurity} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.collection.BitSet @@ -34,6 +33,8 @@ import org.apache.spark.util.collection.BitSet */ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { + //////////////////////////////// Integration tests ////////////////////////////////// + test("run deep example") { val data = Range(0, 3).map(x => LabeledPoint(math.pow(x, 3), Vectors.dense(x))) val df = sqlContext.createDataFrame(data) @@ -70,24 +71,26 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { assert(grandkids.forall(_.isInstanceOf[InternalNode])) } + //////////////////////////////// Helper classes ////////////////////////////////// + test("FeatureVector") { - val v = new FeatureVector(1, FeatureType.Continuous, Array(0.1, 0.3, 0.7), Array(1, 2, 0)) + val v = new FeatureVector(1, 0, Array(0.1, 0.3, 0.7), Array(1, 2, 0)) val vCopy = v.deepCopy() vCopy.values(0) = 1000 assert(v.values(0) !== vCopy.values(0)) val original = Vectors.dense(0.7, 0.1, 0.3) - val v2 = FeatureVector.fromOriginal(1, FeatureType.Continuous, original) + val v2 = FeatureVector.fromOriginal(1, 0, original) assert(v === v2) } test("PartitionInfo") { val numRows = 4 val col1 = - FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.8, 0.2, 0.1, 0.6)) + FeatureVector.fromOriginal(0, 0, Vectors.dense(0.8, 0.2, 0.1, 0.6)) val col2 = - FeatureVector.fromOriginal(1, FeatureType.Categorical, Vectors.dense(0, 1, 0, 2)) + FeatureVector.fromOriginal(1, 3, Vectors.dense(0, 1, 0, 2)) assert(col1.values.length === numRows) assert(col2.values.length === numRows) val nodeOffsets = Array(0, numRows) @@ -106,9 +109,9 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { assert(newInfo.columns.length === 2) val expectedCol1a = - new FeatureVector(0, FeatureType.Continuous, Array(0.1, 0.8, 0.2, 0.6), Array(2, 0, 1, 3)) + new FeatureVector(0, 0, Array(0.1, 0.8, 0.2, 0.6), Array(2, 0, 1, 3)) val expectedCol1b = - new FeatureVector(1, FeatureType.Categorical, Array(0, 0, 1, 2), Array(0, 2, 1, 3)) + new FeatureVector(1, 3, Array(0, 0, 1, 2), Array(0, 2, 1, 3)) assert(newInfo.columns(0) === expectedCol1a) assert(newInfo.columns(1) === expectedCol1b) assert(newInfo.nodeOffsets === Array(0, 2, 4)) @@ -124,18 +127,72 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { assert(newInfo2.columns.length === 2) val expectedCol2a = - new FeatureVector(0, FeatureType.Continuous, Array(0.8, 0.1, 0.2, 0.6), Array(0, 2, 1, 3)) + new FeatureVector(0, 0, Array(0.8, 0.1, 0.2, 0.6), Array(0, 2, 1, 3)) val expectedCol2b = - new FeatureVector(1, FeatureType.Categorical, Array(0, 0, 1, 2), Array(0, 2, 1, 3)) + new FeatureVector(1, 3, Array(0, 0, 1, 2), Array(0, 2, 1, 3)) assert(newInfo2.columns(0) === expectedCol2a) assert(newInfo2.columns(1) === expectedCol2b) assert(newInfo2.nodeOffsets === Array(0, 1, 2, 3, 4)) assert(newInfo2.activeNodes.iterator.toSet === Set(0, 1, 2, 3)) } + //////////////////////////////// Misc ////////////////////////////////// + + test("numUnorderedBins") { + // Note: We have duplicate bins (the inverse) for unordered features. This should be fixed! + assert(AltDT.numUnorderedBins(2) === 2) // 2 categories => 2 bins + assert(AltDT.numUnorderedBins(3) === 6) // 3 categories => 6 bins + } + + //////////////////////////////// Choosing splits ////////////////////////////////// + + test("computeBestSplits") { + } + + test("chooseSplit") { + } + + test("chooseOrderedCategoricalSplit") { + } + + // test("chooseUnorderedCategoricalSplit") { } + + test("chooseContinuousSplit: basic case") { + val featureIndex = 0 + val values = Seq(0.1, 0.2, 0.3, 0.4, 0.5) + val labels = Seq(0.0, 0.0, 1.0, 1.0, 1.0) + val impurity = Entropy + val metadata = new AltDTMetadata(numClasses = 2, maxBins = 4, minInfoGain = 0.0, impurity) + val (split, stats) = AltDT.chooseContinuousSplit(featureIndex, values, labels, metadata) + split match { + case s: ContinuousSplit => + assert(s.featureIndex === featureIndex) + assert(s.threshold === 0.2) + case _ => + throw new AssertionError( + s"Expected ContinuousSplit but got ${split.getClass.getSimpleName}") + } + val fullImpurityStatsArray = Array(2.0, 3.0) + val fullImpurity = impurity.calculate(fullImpurityStatsArray, labels.length) + assert(stats.gain === fullImpurity) + assert(stats.impurity === fullImpurity) + assert(stats.impurityCalculator.stats === fullImpurityStatsArray) + assert(stats.leftImpurityCalculator.stats === Array(2.0, 0.0)) + assert(stats.rightImpurityCalculator.stats === Array(0.0, 3.0)) + assert(stats.valid) + } + + test("chooseContinuousSplit: some equal values") { + } + + // TODO: Add this test once we make this change. + // test("chooseContinuousSplit: return bad split if best split is on end") { } + + //////////////////////////////// Bit subvectors ////////////////////////////////// + test("bitSubvectorFromSplit: 1 node") { val col = - FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) + FeatureVector.fromOriginal(0, 0, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) val fromOffset = 0 val toOffset = col.values.length val split = new ContinuousSplit(0, threshold = 0.5) @@ -147,7 +204,7 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { test("bitSubvectorFromSplit: 2 nodes") { // Initially, 1 split: (0, 2, 4) | (1, 3) - val col = new FeatureVector(0, FeatureType.Continuous, Array(0.1, 0.2, 0.4, 0.6, 0.7), + val col = new FeatureVector(0, 0, Array(0.1, 0.2, 0.4, 0.6, 0.7), Array(4, 2, 0, 1, 3)) def checkSplit(fromOffset: Int, toOffset: Int, threshold: Double, expectedRight: Set[Int]): Unit = { val split = new ContinuousSplit(0, threshold) @@ -166,24 +223,16 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { checkSplit(3, 5, 0.8, Set()) } - test("computeBestSplits") { - - } - - test("computeActiveNodePeriphery") { - } - test("collectBitVectors with 1 vector") { val col = - FeatureVector.fromOriginal(0, FeatureType.Continuous, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) + FeatureVector.fromOriginal(0, 0, Vectors.dense(0.1, 0.2, 0.4, 0.6, 0.7)) val numRows = col.values.length val activeNodes = new BitSet(1) activeNodes.set(0) val info = PartitionInfo(Array(col), Array(0, numRows), activeNodes) val partitionInfos = sc.parallelize(Seq(info)) - val bestSplitAndGain = (new ContinuousSplit(0, threshold = 0.5), - new InfoGainStats(0, 0, 0, 0, 0, new Predict(0, 0), new Predict(0, 0))) - val bitVectors = AltDT.collectBitVectors(partitionInfos, Array(bestSplitAndGain)) + val bestSplit = new ContinuousSplit(0, threshold = 0.5) + val bitVectors = AltDT.collectBitVectors(partitionInfos, Array(bestSplit)) assert(bitVectors.length === 1) val bitv = bitVectors.head assert(bitv.numBits === numRows) @@ -191,20 +240,24 @@ class AltDTSuite extends SparkFunSuite with MLlibTestSparkContext { } test("collectBitVectors with 1 vector, with tied threshold") { - val col = new FeatureVector(0, FeatureType.Continuous, + val col = new FeatureVector(0, 0, Array(-4.0,-4.0,-2.0,-2.0,-1.0,-1.0,1.0,1.0), Array(3,7,2,6,1,5,0,4)) val numRows = col.values.length val activeNodes = new BitSet(1) activeNodes.set(0) val info = PartitionInfo(Array(col), Array(0, numRows), activeNodes) val partitionInfos = sc.parallelize(Seq(info)) - val bestSplitAndGain = (new ContinuousSplit(0, threshold = -2.0), - new InfoGainStats(0, 0, 0, 0, 0, new Predict(0, 0), new Predict(0, 0))) - val bitVectors = AltDT.collectBitVectors(partitionInfos, Array(bestSplitAndGain)) + val bestSplit = new ContinuousSplit(0, threshold = -2.0) + val bitVectors = AltDT.collectBitVectors(partitionInfos, Array(bestSplit)) assert(bitVectors.length === 1) val bitv = bitVectors.head assert(bitv.numBits === numRows) assert(bitv.iterator.toArray === Array(0, 1, 4, 5)) } + //////////////////////////////// Active nodes ////////////////////////////////// + + test("computeActiveNodePeriphery") { + } + } From d897714e9a52de1e569a73b2eb829355ec9143fe Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 29 Sep 2015 12:40:37 +0100 Subject: [PATCH 20/34] Changes BitSubvector to use System.arraycopy --- .../apache/spark/util/collection/BitSet.scala | 10 +++++ .../spark/util/collection/BitSetSuite.scala | 37 +++++++++++++------ .../spark/ml/tree/impl/BitSubvector.scala | 11 +++++- .../ml/tree/impl/BitSubvectorSuite.scala | 6 +++ 4 files changed, 52 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 9c15b1188d91..cf5dbd2405da 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -46,6 +46,16 @@ class BitSet(numBits: Int) extends Serializable { } } + /** + * Copies the bits from another bitset into this bitset, mutating this + * bitset in-place. This method assumes both bitsets have the same + * numWords. + */ + def copyFrom(other: BitSet): Unit = { + require(numWords == other.numWords) + System.arraycopy(other.words, 0, words, 0, numWords) + } + /** * Compute the bit-wise AND of the two sets returning the * result. diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 69dbfa9cd714..e717cdc06cf4 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -21,24 +21,39 @@ import org.apache.spark.SparkFunSuite class BitSetSuite extends SparkFunSuite { - test("basic set and get") { + test("basic routines") { val setBits = Seq(0, 9, 1, 10, 90, 96) val bitset = new BitSet(100) + val copyBitset = new BitSet(100) - for (i <- 0 until 100) { - assert(!bitset.get(i)) - } + test("set and get") { + for (i <- 0 until 100) { + assert(!bitset.get(i)) + } - setBits.foreach(i => bitset.set(i)) + setBits.foreach(i => bitset.set(i)) - for (i <- 0 until 100) { - if (setBits.contains(i)) { - assert(bitset.get(i)) - } else { - assert(!bitset.get(i)) + for (i <- 0 until 100) { + if (setBits.contains(i)) { + assert(bitset.get(i)) + } else { + assert(!bitset.get(i)) + } + } + assert(bitset.cardinality() === setBits.size) + } + + test("copyFrom") { + copyBitset.copyFrom(bitset) + for (i <- 0 until 100) { + if (setBits.contains(i)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) + } } + assert(copyBitset.cardinality() === setBits.size) } - assert(bitset.cardinality() === setBits.size) } test("100% full bit set") { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index fe9b171cfd1b..81f8c2109263 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -41,6 +41,14 @@ private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializabl override def hasNext: Boolean = iter.hasNext override def next(): Int = iter.next() - offset + from } + + /** + * Copies the bits from another BitSubvector, mutating the BitSubvector in-place. This method assumes both + * BitSubvectors have the same from and to. + */ + def copyBitsFrom(other: BitSubvector): Unit = { + bits.copyFrom(other.bits) + } } private[impl] object BitSubvector { @@ -70,7 +78,8 @@ private[impl] object BitSubvector { if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 val newSubv = newSubvectors(curNewSubvIdx) // TODO: More efficient (word-level) copy. - subv.iterator.foreach(idx => newSubv.set(idx)) + // subv.iterator.foreach(idx => newSubv.set(idx)) + newSubv.copyBitsFrom(subv) } assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check newSubvectors diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index 6e7b67623851..92a42b75a941 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -37,6 +37,12 @@ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(bs.get(x)) } assert(bs.iterator.toSet === setVals.toSet) + + test("copy") { + val copyBs = new BitSubvector(from, to) + copyBs.copyBitsFrom(bs) + assert(copyBs.iterator.toSet === setVals.toSet) + } } test("BitSubvector merge") { From 6cc1ed89744360dd3702a65a102246880925e7b3 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 29 Sep 2015 12:49:12 +0100 Subject: [PATCH 21/34] Removes commented code --- .../main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index 81f8c2109263..62dc7a498ab0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -78,7 +78,6 @@ private[impl] object BitSubvector { if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 val newSubv = newSubvectors(curNewSubvIdx) // TODO: More efficient (word-level) copy. - // subv.iterator.foreach(idx => newSubv.set(idx)) newSubv.copyBitsFrom(subv) } assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check From 71fd54cc41775163622f6526c9896bdc80f05ae8 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 29 Sep 2015 18:03:31 +0100 Subject: [PATCH 22/34] Fixes nested tests --- .../spark/util/collection/BitSetSuite.scala | 4 ++-- .../ml/tree/impl/BitSubvectorSuite.scala | 23 ++++++++++++------- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index e717cdc06cf4..1bd43b32dbc7 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite class BitSetSuite extends SparkFunSuite { - test("basic routines") { + (() => { val setBits = Seq(0, 9, 1, 10, 90, 96) val bitset = new BitSet(100) val copyBitset = new BitSet(100) @@ -54,7 +54,7 @@ class BitSetSuite extends SparkFunSuite { } assert(copyBitset.cardinality() === setBits.size) } - } + })() test("100% full bit set") { val bitset = new BitSet(10000) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index 92a42b75a941..aaf76c886c80 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -25,25 +25,32 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext */ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { - test("BitSubvector basic ops") { + (() => { val from = 1 val to = 4 val bs = new BitSubvector(from, to) - assert(bs.numBits === to - from) - Range(from, to).foreach(x => assert(!bs.get(x))) val setVals = Array(from, to - 1) - setVals.foreach { x => - bs.set(x) - assert(bs.get(x)) + + test("basic ops") { + assert(bs.numBits === to - from) + Range(from, to).foreach(x => assert(!bs.get(x))) + setVals.foreach { x => + bs.set(x) + assert(bs.get(x)) + } + assert(bs.iterator.toSet === setVals.toSet) } - assert(bs.iterator.toSet === setVals.toSet) test("copy") { val copyBs = new BitSubvector(from, to) + setVals.foreach { x => + bs.set(x) + assert(bs.get(x)) + } copyBs.copyBitsFrom(bs) assert(copyBs.iterator.toSet === setVals.toSet) } - } + })() test("BitSubvector merge") { val b1 = new BitSubvector(0, 5) From 1597d113b595e0bc8e0a9c0384c924b9d763e8b4 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 3 Oct 2015 14:46:40 +0100 Subject: [PATCH 23/34] Makes BitSubvectors left-align on word boundaries and adds comments --- .../spark/ml/tree/impl/BitSubvector.scala | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index 62dc7a498ab0..05ba1811715d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -22,24 +22,27 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.collection.BitSet +/** + * A range of bits within a larger distributed bit vector. + * @param from starting index (inclusive) of larger distributed bit vector represented by this instance + * @param to ending index (exclusive) of larger distributed bit vector represented by this instance + */ private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializable { val numBits: Int = to - from - /** Element i will be put at location i + offset in the BitSet */ - private val offset: Int = 64 - (numBits % 64) - - private val bits: BitSet = new BitSet(numBits + offset) + private val bits: BitSet = new BitSet(numBits) - def set(bit: Int): Unit = bits.set(bit + offset - from) + /** Set a bit in this instance using an external index */ + def set(idx: Int): Unit = bits.set(toInternalIdx(idx)) - def get(bit: Int): Boolean = bits.get(bit + offset - from) + def get(idx: Int): Boolean = bits.get(toInternalIdx(idx)) - /** Get an iterator over the set bits. */ + /** Get an iterator over the external indices of the set bits. */ def iterator: Iterator[Int] = new Iterator[Int] { val iter = bits.iterator override def hasNext: Boolean = iter.hasNext - override def next(): Int = iter.next() - offset + from + override def next(): Int = toExternalIdx(iter.next()) } /** @@ -49,6 +52,14 @@ private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializabl def copyBitsFrom(other: BitSubvector): Unit = { bits.copyFrom(other.bits) } + + private def toInternalIdx(idx: Int): Int = { + require(idx >= from && idx < to) + idx - from + } + private def toExternalIdx(idx: Int): Int = { + idx + from + } } private[impl] object BitSubvector { @@ -77,8 +88,9 @@ private[impl] object BitSubvector { sortedSubvectors.foreach { subv => if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 val newSubv = newSubvectors(curNewSubvIdx) - // TODO: More efficient (word-level) copy. - newSubv.copyBitsFrom(subv) + // TODO: More efficient (word-level) copy.> +// newSubv.copyBitsFrom(subv) + subv.iterator.foreach(idx => newSubv.set(idx)) } assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check newSubvectors From 92a6fa56dc12666eb2a0104f038e67062c4448e3 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 3 Oct 2015 15:15:21 +0100 Subject: [PATCH 24/34] Changes copyFrom to orWithOffset --- .../org/apache/spark/util/collection/BitSet.scala | 14 ++++---------- .../apache/spark/util/collection/BitSetSuite.scala | 4 ++-- .../apache/spark/ml/tree/impl/BitSubvector.scala | 13 ++++++++----- .../spark/ml/tree/impl/BitSubvectorSuite.scala | 4 ++-- 4 files changed, 16 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index cf5dbd2405da..db83984ae477 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -46,16 +46,6 @@ class BitSet(numBits: Int) extends Serializable { } } - /** - * Copies the bits from another bitset into this bitset, mutating this - * bitset in-place. This method assumes both bitsets have the same - * numWords. - */ - def copyFrom(other: BitSet): Unit = { - require(numWords == other.numWords) - System.arraycopy(other.words, 0, words, 0, numWords) - } - /** * Compute the bit-wise AND of the two sets returning the * result. @@ -229,4 +219,8 @@ class BitSet(numBits: Int) extends Serializable { /** Return the number of longs it would take to hold numBits. */ private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 + + private[spark] def orWithOffset(other: BitSet, offset: Int, numBits: Int): Unit = { + ??? + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 1bd43b32dbc7..39c4d4e57f32 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -43,8 +43,8 @@ class BitSetSuite extends SparkFunSuite { assert(bitset.cardinality() === setBits.size) } - test("copyFrom") { - copyBitset.copyFrom(bitset) + test("orWithOffset") { + copyBitset.orWithOffset(bitset, 0, bitset.capacity / 64) for (i <- 0 until 100) { if (setBits.contains(i)) { assert(copyBitset.get(i)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index 05ba1811715d..6d7d16c68cb4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -46,11 +46,13 @@ private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializabl } /** - * Copies the bits from another BitSubvector, mutating the BitSubvector in-place. This method assumes both - * BitSubvectors have the same from and to. + * Bit-wise OR with another BitSubvector. Bits are matched according to external index (ORed against 0 if absent in + * the other BitSubvector). This method mutates the current instance in-place. */ - def copyBitsFrom(other: BitSubvector): Unit = { - bits.copyFrom(other.bits) + def |=(other: BitSubvector): Unit = { + require(from <= other.from && to >= other.to) + val delta = other.from - from + bits.orWithOffset(other.bits, delta, math.min(numBits, other.numBits - delta)) } private def toInternalIdx(idx: Int): Int = { @@ -90,7 +92,8 @@ private[impl] object BitSubvector { val newSubv = newSubvectors(curNewSubvIdx) // TODO: More efficient (word-level) copy.> // newSubv.copyBitsFrom(subv) - subv.iterator.foreach(idx => newSubv.set(idx)) +// subv.iterator.foreach(idx => newSubv.set(idx)) + newSubv |= subv } assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check newSubvectors diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index aaf76c886c80..7d853257f23c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -41,13 +41,13 @@ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(bs.iterator.toSet === setVals.toSet) } - test("copy") { + test("|=") { val copyBs = new BitSubvector(from, to) setVals.foreach { x => bs.set(x) assert(bs.get(x)) } - copyBs.copyBitsFrom(bs) + copyBs |= bs assert(copyBs.iterator.toSet === setVals.toSet) } })() From 506ac1004b852a6973b62a8e99fca0b6bc3a2cc6 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 3 Oct 2015 15:23:13 +0100 Subject: [PATCH 25/34] Adds dumb impl and failing tests for offset!=0 --- .../apache/spark/util/collection/BitSet.scala | 3 +- .../spark/util/collection/BitSetSuite.scala | 33 +++++++++++++++++-- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index db83984ae477..40755c84d167 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -220,7 +220,8 @@ class BitSet(numBits: Int) extends Serializable { /** Return the number of longs it would take to hold numBits. */ private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 + /** Shifts other left by offset bits before ORing with this instance and mutating in-place. */ private[spark] def orWithOffset(other: BitSet, offset: Int, numBits: Int): Unit = { - ??? + (this | other).words.zipWithIndex.map { case (v,i) => words(i) = v } } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 39c4d4e57f32..ba1878677968 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -24,7 +24,6 @@ class BitSetSuite extends SparkFunSuite { (() => { val setBits = Seq(0, 9, 1, 10, 90, 96) val bitset = new BitSet(100) - val copyBitset = new BitSet(100) test("set and get") { for (i <- 0 until 100) { @@ -43,8 +42,9 @@ class BitSetSuite extends SparkFunSuite { assert(bitset.cardinality() === setBits.size) } - test("orWithOffset") { - copyBitset.orWithOffset(bitset, 0, bitset.capacity / 64) + test("orWithOffset offset=0") { + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, 0, bitset.capacity) for (i <- 0 until 100) { if (setBits.contains(i)) { assert(copyBitset.get(i)) @@ -54,6 +54,33 @@ class BitSetSuite extends SparkFunSuite { } assert(copyBitset.cardinality() === setBits.size) } + + test("orWithOffset offset=5") { + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, 5, bitset.capacity - 5) + for (i <- 0 until (100 - 5)) { + if (setBits.contains(i + 5)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) + } + } + assert(copyBitset.cardinality() === setBits.size) + } + + + test("orWithOffset offset=65 (full word + 1)") { + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, 65, bitset.capacity - 65) + for (i <- 0 until (100 - 65)) { + if (setBits.contains(i + 65)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) + } + } + assert(copyBitset.cardinality() === setBits.size) + } })() test("100% full bit set") { From 2c329df1058e2c716b15c40182dd65c54e3101c4 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 3 Oct 2015 17:06:37 +0100 Subject: [PATCH 26/34] Pass unit tests, adds debug code --- .../apache/spark/util/collection/BitSet.scala | 26 ++++++++++++++++++- .../spark/util/collection/BitSetSuite.scala | 21 +++++++-------- 2 files changed, 35 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 40755c84d167..e6f6bb5b6099 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -221,7 +221,31 @@ class BitSet(numBits: Int) extends Serializable { private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 /** Shifts other left by offset bits before ORing with this instance and mutating in-place. */ + // TODO: Add offset in original as well in case the two "from"s are not equal private[spark] def orWithOffset(other: BitSet, offset: Int, numBits: Int): Unit = { - (this | other).words.zipWithIndex.map { case (v,i) => words(i) = v } + // Bit vectors have memory layout [63..0|127..64|...] where | denotes word boundaries + var wordIndex = 0 + val numWords = bit2words(numBits) + val wordOffset = offset >> 6 + val bitOffset = offset % 64 + println(s"numWords: $numWords, wordOffset: $wordOffset, bitOffset: $bitOffset") + println(s"this: ${words.toList.map(_.toBinaryString)}, other: ${other.words.toList.map(_.toBinaryString)}") + while (wordIndex < numWords && wordIndex + wordOffset < other.numWords) { + val maskedShiftedOtherWord = (other.words(wordIndex + wordOffset) & (Long.MaxValue << bitOffset)) >> bitOffset + println(s"wordIndex: $wordIndex") + println(s"ORing ${words(wordIndex).toBinaryString} with ${maskedShiftedOtherWord.toBinaryString}") + words(wordIndex) = words(wordIndex) | maskedShiftedOtherWord + + if (bitOffset > 0 && wordIndex + wordOffset + 1 < other.numWords) { + val leftOffset = 64 - bitOffset + val maskedShiftedNextWord = (other.words(wordIndex + wordOffset + 1) & (Long.MaxValue >> leftOffset)) << leftOffset + println( + s"ORing ${words(wordIndex).toBinaryString} " + + s"with ${maskedShiftedNextWord.toBinaryString}") + words(wordIndex) = words(wordIndex) | maskedShiftedNextWord + } + wordIndex += 1 + } + println(s"ending with ${words.toList.map(_.toBinaryString)}") } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index ba1878677968..962db86ffda2 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -58,28 +58,27 @@ class BitSetSuite extends SparkFunSuite { test("orWithOffset offset=5") { val copyBitset = new BitSet(100) copyBitset.orWithOffset(bitset, 5, bitset.capacity - 5) - for (i <- 0 until (100 - 5)) { - if (setBits.contains(i + 5)) { - assert(copyBitset.get(i)) + for (i <- 5 until 100) { + if (setBits.contains(i)) { + assert(copyBitset.get(i - 5)) } else { - assert(!copyBitset.get(i)) + assert(!copyBitset.get(i - 5)) } } - assert(copyBitset.cardinality() === setBits.size) } - test("orWithOffset offset=65 (full word + 1)") { val copyBitset = new BitSet(100) copyBitset.orWithOffset(bitset, 65, bitset.capacity - 65) - for (i <- 0 until (100 - 65)) { - if (setBits.contains(i + 65)) { - assert(copyBitset.get(i)) + println(copyBitset.iterator.toList) + println(setBits.iterator.toList) + for (i <- 65 until 100) { + if (setBits.contains(i)) { + assert(copyBitset.get(i - 65)) } else { - assert(!copyBitset.get(i)) + assert(!copyBitset.get(i - 65)) } } - assert(copyBitset.cardinality() === setBits.size) } })() From 9a642b86b20a30c58b39777a22a206fbba0acbb9 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 3 Oct 2015 17:09:18 +0100 Subject: [PATCH 27/34] Removes debug printlns --- .../org/apache/spark/util/collection/BitSet.scala | 11 +++-------- .../apache/spark/util/collection/BitSetSuite.scala | 2 -- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index e6f6bb5b6099..23d059cf1d7a 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -228,24 +228,19 @@ class BitSet(numBits: Int) extends Serializable { val numWords = bit2words(numBits) val wordOffset = offset >> 6 val bitOffset = offset % 64 - println(s"numWords: $numWords, wordOffset: $wordOffset, bitOffset: $bitOffset") - println(s"this: ${words.toList.map(_.toBinaryString)}, other: ${other.words.toList.map(_.toBinaryString)}") while (wordIndex < numWords && wordIndex + wordOffset < other.numWords) { + // Mask, shift, and OR with current word val maskedShiftedOtherWord = (other.words(wordIndex + wordOffset) & (Long.MaxValue << bitOffset)) >> bitOffset - println(s"wordIndex: $wordIndex") - println(s"ORing ${words(wordIndex).toBinaryString} with ${maskedShiftedOtherWord.toBinaryString}") words(wordIndex) = words(wordIndex) | maskedShiftedOtherWord + // Fill in higher-order bits from next word if available if (bitOffset > 0 && wordIndex + wordOffset + 1 < other.numWords) { val leftOffset = 64 - bitOffset val maskedShiftedNextWord = (other.words(wordIndex + wordOffset + 1) & (Long.MaxValue >> leftOffset)) << leftOffset - println( - s"ORing ${words(wordIndex).toBinaryString} " + - s"with ${maskedShiftedNextWord.toBinaryString}") words(wordIndex) = words(wordIndex) | maskedShiftedNextWord } + wordIndex += 1 } - println(s"ending with ${words.toList.map(_.toBinaryString)}") } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 962db86ffda2..986dce14bbc5 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -70,8 +70,6 @@ class BitSetSuite extends SparkFunSuite { test("orWithOffset offset=65 (full word + 1)") { val copyBitset = new BitSet(100) copyBitset.orWithOffset(bitset, 65, bitset.capacity - 65) - println(copyBitset.iterator.toList) - println(setBits.iterator.toList) for (i <- 65 until 100) { if (setBits.contains(i)) { assert(copyBitset.get(i - 65)) From 49a628a534ae598bf34ce5c8c863681574f75ce8 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 3 Oct 2015 18:05:06 +0100 Subject: [PATCH 28/34] Fixes implementation, cleans comments, more tests --- .../apache/spark/util/collection/BitSet.scala | 34 ++++++++++--------- .../spark/util/collection/BitSetSuite.scala | 12 +++---- .../spark/ml/tree/impl/BitSubvector.scala | 5 +-- .../ml/tree/impl/BitSubvectorSuite.scala | 4 --- 4 files changed, 25 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 23d059cf1d7a..fb9948e124cb 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -220,26 +220,28 @@ class BitSet(numBits: Int) extends Serializable { /** Return the number of longs it would take to hold numBits. */ private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 - /** Shifts other left by offset bits before ORing with this instance and mutating in-place. */ - // TODO: Add offset in original as well in case the two "from"s are not equal + /** Bit-wise OR between two BitSets where the ith bit of other is ORed against the i+offset bit of this instance. */ private[spark] def orWithOffset(other: BitSet, offset: Int, numBits: Int): Unit = { - // Bit vectors have memory layout [63..0|127..64|...] where | denotes word boundaries - var wordIndex = 0 val numWords = bit2words(numBits) - val wordOffset = offset >> 6 - val bitOffset = offset % 64 - while (wordIndex < numWords && wordIndex + wordOffset < other.numWords) { - // Mask, shift, and OR with current word - val maskedShiftedOtherWord = (other.words(wordIndex + wordOffset) & (Long.MaxValue << bitOffset)) >> bitOffset - words(wordIndex) = words(wordIndex) | maskedShiftedOtherWord - - // Fill in higher-order bits from next word if available - if (bitOffset > 0 && wordIndex + wordOffset + 1 < other.numWords) { - val leftOffset = 64 - bitOffset - val maskedShiftedNextWord = (other.words(wordIndex + wordOffset + 1) & (Long.MaxValue >> leftOffset)) << leftOffset - words(wordIndex) = words(wordIndex) | maskedShiftedNextWord + val wordOffset = offset >> 6 // divide by 64 + + // Bit vectors have memory layout [63..0|127..64|...] where | denotes word boundaries, so left/right within a word + // and left/right across words are flipped + val rightOffset = offset % 64 + val leftOffset = 64 - rightOffset + + var wordIndex = 0 + while (wordIndex < numWords) { + // Fill in lowest-order bits from other's previous word's highest-order bits if available + if (rightOffset > 0 && wordIndex > 0) { + val maskedShiftedPrevWord = (other.words(wordIndex - 1) & (Long.MaxValue << leftOffset)) >> leftOffset + words(wordIndex + wordOffset) = words(wordIndex + wordOffset) | maskedShiftedPrevWord } + // Mask, shift, and OR with current word + val maskedShiftedOtherWord = (other.words(wordIndex) & (Long.MaxValue >> rightOffset)) << rightOffset + words(wordIndex + wordOffset) = words(wordIndex + wordOffset) | maskedShiftedOtherWord + wordIndex += 1 } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 986dce14bbc5..362c12eb9687 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -59,10 +59,10 @@ class BitSetSuite extends SparkFunSuite { val copyBitset = new BitSet(100) copyBitset.orWithOffset(bitset, 5, bitset.capacity - 5) for (i <- 5 until 100) { - if (setBits.contains(i)) { - assert(copyBitset.get(i - 5)) + if (setBits.contains(i - 5)) { + assert(copyBitset.get(i)) } else { - assert(!copyBitset.get(i - 5)) + assert(!copyBitset.get(i)) } } } @@ -71,10 +71,10 @@ class BitSetSuite extends SparkFunSuite { val copyBitset = new BitSet(100) copyBitset.orWithOffset(bitset, 65, bitset.capacity - 65) for (i <- 65 until 100) { - if (setBits.contains(i)) { - assert(copyBitset.get(i - 65)) + if (setBits.contains(i - 65)) { + assert(copyBitset.get(i)) } else { - assert(!copyBitset.get(i - 65)) + assert(!copyBitset.get(i)) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index 6d7d16c68cb4..7a95e659c8e8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -52,7 +52,7 @@ private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializabl def |=(other: BitSubvector): Unit = { require(from <= other.from && to >= other.to) val delta = other.from - from - bits.orWithOffset(other.bits, delta, math.min(numBits, other.numBits - delta)) + bits.orWithOffset(other.bits, delta, math.min(numBits - delta, other.numBits)) } private def toInternalIdx(idx: Int): Int = { @@ -90,9 +90,6 @@ private[impl] object BitSubvector { sortedSubvectors.foreach { subv => if (subv.to > newSubvectors(curNewSubvIdx).to) curNewSubvIdx += 1 val newSubv = newSubvectors(curNewSubvIdx) - // TODO: More efficient (word-level) copy.> -// newSubv.copyBitsFrom(subv) -// subv.iterator.foreach(idx => newSubv.set(idx)) newSubv |= subv } assert(curNewSubvIdx + 1 == newSubvectors.length) // sanity check diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index 7d853257f23c..eac75b17d66b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -43,10 +43,6 @@ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { test("|=") { val copyBs = new BitSubvector(from, to) - setVals.foreach { x => - bs.set(x) - assert(bs.get(x)) - } copyBs |= bs assert(copyBs.iterator.toSet === setVals.toSet) } From c5e54804abbfa35c82584b84a998bb1783e87d83 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 10 Oct 2015 23:21:21 +0100 Subject: [PATCH 29/34] Improves docs and fixes Long.MaxValue bug --- .../apache/spark/util/collection/BitSet.scala | 21 ++++++++++++------- .../spark/util/collection/BitSetSuite.scala | 6 +++--- .../spark/ml/tree/impl/BitSubvector.scala | 2 +- 3 files changed, 18 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index fb9948e124cb..bc94433f7a27 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -220,26 +220,33 @@ class BitSet(numBits: Int) extends Serializable { /** Return the number of longs it would take to hold numBits. */ private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 - /** Bit-wise OR between two BitSets where the ith bit of other is ORed against the i+offset bit of this instance. */ - private[spark] def orWithOffset(other: BitSet, offset: Int, numBits: Int): Unit = { - val numWords = bit2words(numBits) + /** + * Bit-wise OR between two BitSets where the ith bit of other is ORed against the i+offset bit of this instance. For + * performance, the OR is computed word-by-word rather than bit-by-bit. + * + * This function mutates the current BitSet instance (i.e. not `other`). + * + * @param offset the amount to left-shift (with zero padding) `other` before performing the OR, must be >= 0. + */ + private[spark] def orWithOffset(other: BitSet, offset: Int): Unit = { + val numWords = bit2words(math.min(this.capacity, other.capacity - offset)) val wordOffset = offset >> 6 // divide by 64 // Bit vectors have memory layout [63..0|127..64|...] where | denotes word boundaries, so left/right within a word // and left/right across words are flipped - val rightOffset = offset % 64 - val leftOffset = 64 - rightOffset + val rightOffset = offset & 0x3f // mod 64 + val leftOffset = (64 - rightOffset) & 0x3f // mod 64 var wordIndex = 0 while (wordIndex < numWords) { // Fill in lowest-order bits from other's previous word's highest-order bits if available if (rightOffset > 0 && wordIndex > 0) { - val maskedShiftedPrevWord = (other.words(wordIndex - 1) & (Long.MaxValue << leftOffset)) >> leftOffset + val maskedShiftedPrevWord = (other.words(wordIndex - 1) & (-1L << leftOffset)) >> leftOffset words(wordIndex + wordOffset) = words(wordIndex + wordOffset) | maskedShiftedPrevWord } // Mask, shift, and OR with current word - val maskedShiftedOtherWord = (other.words(wordIndex) & (Long.MaxValue >> rightOffset)) << rightOffset + val maskedShiftedOtherWord = (other.words(wordIndex) & (-1L >> rightOffset)) << rightOffset words(wordIndex + wordOffset) = words(wordIndex + wordOffset) | maskedShiftedOtherWord wordIndex += 1 diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 362c12eb9687..a2ff00d86ce9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -44,7 +44,7 @@ class BitSetSuite extends SparkFunSuite { test("orWithOffset offset=0") { val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 0, bitset.capacity) + copyBitset.orWithOffset(bitset, 0) for (i <- 0 until 100) { if (setBits.contains(i)) { assert(copyBitset.get(i)) @@ -57,7 +57,7 @@ class BitSetSuite extends SparkFunSuite { test("orWithOffset offset=5") { val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 5, bitset.capacity - 5) + copyBitset.orWithOffset(bitset, 5) for (i <- 5 until 100) { if (setBits.contains(i - 5)) { assert(copyBitset.get(i)) @@ -69,7 +69,7 @@ class BitSetSuite extends SparkFunSuite { test("orWithOffset offset=65 (full word + 1)") { val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 65, bitset.capacity - 65) + copyBitset.orWithOffset(bitset, 65) for (i <- 65 until 100) { if (setBits.contains(i - 65)) { assert(copyBitset.get(i)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala index 7a95e659c8e8..b94f4903dba4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BitSubvector.scala @@ -52,7 +52,7 @@ private[impl] class BitSubvector(val from: Int, val to: Int) extends Serializabl def |=(other: BitSubvector): Unit = { require(from <= other.from && to >= other.to) val delta = other.from - from - bits.orWithOffset(other.bits, delta, math.min(numBits - delta, other.numBits)) + bits.orWithOffset(other.bits, delta) } private def toInternalIdx(idx: Int): Int = { From 1213f86f0d295a2735da44c5fc7766ae0c01e8d5 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 10 Oct 2015 23:27:33 +0100 Subject: [PATCH 30/34] Cleans up shared state in BitSetSuite --- .../spark/util/collection/BitSetSuite.scala | 94 ++++++++++--------- 1 file changed, 52 insertions(+), 42 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index a2ff00d86ce9..e951f6f1c289 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -21,64 +21,74 @@ import org.apache.spark.SparkFunSuite class BitSetSuite extends SparkFunSuite { - (() => { + test("set and get") { val setBits = Seq(0, 9, 1, 10, 90, 96) val bitset = new BitSet(100) - test("set and get") { - for (i <- 0 until 100) { - assert(!bitset.get(i)) - } + for (i <- 0 until 100) { + assert(!bitset.get(i)) + } - setBits.foreach(i => bitset.set(i)) + setBits.foreach(i => bitset.set(i)) - for (i <- 0 until 100) { - if (setBits.contains(i)) { - assert(bitset.get(i)) - } else { - assert(!bitset.get(i)) - } + for (i <- 0 until 100) { + if (setBits.contains(i)) { + assert(bitset.get(i)) + } else { + assert(!bitset.get(i)) } - assert(bitset.cardinality() === setBits.size) } + assert(bitset.cardinality() === setBits.size) + } + + test("orWithOffset offset=0") { + val setBits = Seq(0, 9, 1, 10, 90, 96) + val bitset = new BitSet(100) + setBits.foreach(i => bitset.set(i)) - test("orWithOffset offset=0") { - val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 0) - for (i <- 0 until 100) { - if (setBits.contains(i)) { - assert(copyBitset.get(i)) - } else { - assert(!copyBitset.get(i)) - } + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, 0) + for (i <- 0 until 100) { + if (setBits.contains(i)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) } - assert(copyBitset.cardinality() === setBits.size) } + assert(copyBitset.cardinality() === setBits.size) + } + + test("orWithOffset offset=5") { + val setBits = Seq(0, 9, 1, 10, 90, 96) + val bitset = new BitSet(100) + setBits.foreach(i => bitset.set(i)) - test("orWithOffset offset=5") { - val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 5) - for (i <- 5 until 100) { - if (setBits.contains(i - 5)) { - assert(copyBitset.get(i)) - } else { - assert(!copyBitset.get(i)) - } + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, 5) + for (i <- 5 until 100) { + if (setBits.contains(i - 5)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) } } + } - test("orWithOffset offset=65 (full word + 1)") { - val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 65) - for (i <- 65 until 100) { - if (setBits.contains(i - 65)) { - assert(copyBitset.get(i)) - } else { - assert(!copyBitset.get(i)) - } + test("orWithOffset offset=65 (full word + 1)") { + val setBits = Seq(0, 9, 1, 10, 90, 96) + val bitset = new BitSet(100) + setBits.foreach(i => bitset.set(i)) + + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, 65) + for (i <- 65 until 100) { + if (setBits.contains(i - 65)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) } } - })() + } test("100% full bit set") { val bitset = new BitSet(10000) From 8f1360f396839f8b005c0af5edbc3febccabb128 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 10 Oct 2015 23:30:24 +0100 Subject: [PATCH 31/34] Cleans up shared state in BitSubvectorSuite --- .../ml/tree/impl/BitSubvectorSuite.scala | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index eac75b17d66b..1e2fe6bbd4bf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -25,28 +25,32 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext */ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { - (() => { + test("basic ops") { val from = 1 val to = 4 val bs = new BitSubvector(from, to) val setVals = Array(from, to - 1) - test("basic ops") { - assert(bs.numBits === to - from) - Range(from, to).foreach(x => assert(!bs.get(x))) - setVals.foreach { x => - bs.set(x) - assert(bs.get(x)) - } - assert(bs.iterator.toSet === setVals.toSet) + assert(bs.numBits === to - from) + Range(from, to).foreach(x => assert(!bs.get(x))) + setVals.foreach { x => + bs.set(x) + assert(bs.get(x)) } + assert(bs.iterator.toSet === setVals.toSet) + } - test("|=") { - val copyBs = new BitSubvector(from, to) - copyBs |= bs - assert(copyBs.iterator.toSet === setVals.toSet) - } - })() + test("|=") { + val from = 1 + val to = 4 + val bs = new BitSubvector(from, to) + val setVals = Array(from, to - 1) + setVals.foreach(i => bs.set(i)) + + val copyBs = new BitSubvector(from, to) + copyBs |= bs + assert(copyBs.iterator.toSet === setVals.toSet) + } test("BitSubvector merge") { val b1 = new BitSubvector(0, 5) From 3c5060d91e9335de4872885334bb9200e4dbeb9b Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 10 Oct 2015 23:34:59 +0100 Subject: [PATCH 32/34] Loops through offsets for orWithOffset test --- .../spark/util/collection/BitSetSuite.scala | 53 +++++-------------- 1 file changed, 12 insertions(+), 41 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index e951f6f1c289..3d7f76f6771f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -41,51 +41,22 @@ class BitSetSuite extends SparkFunSuite { assert(bitset.cardinality() === setBits.size) } - test("orWithOffset offset=0") { + test("orWithOffset") { val setBits = Seq(0, 9, 1, 10, 90, 96) val bitset = new BitSet(100) setBits.foreach(i => bitset.set(i)) - val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 0) - for (i <- 0 until 100) { - if (setBits.contains(i)) { - assert(copyBitset.get(i)) - } else { - assert(!copyBitset.get(i)) - } - } - assert(copyBitset.cardinality() === setBits.size) - } - - test("orWithOffset offset=5") { - val setBits = Seq(0, 9, 1, 10, 90, 96) - val bitset = new BitSet(100) - setBits.foreach(i => bitset.set(i)) - - val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 5) - for (i <- 5 until 100) { - if (setBits.contains(i - 5)) { - assert(copyBitset.get(i)) - } else { - assert(!copyBitset.get(i)) - } - } - } - - test("orWithOffset offset=65 (full word + 1)") { - val setBits = Seq(0, 9, 1, 10, 90, 96) - val bitset = new BitSet(100) - setBits.foreach(i => bitset.set(i)) - - val copyBitset = new BitSet(100) - copyBitset.orWithOffset(bitset, 65) - for (i <- 65 until 100) { - if (setBits.contains(i - 65)) { - assert(copyBitset.get(i)) - } else { - assert(!copyBitset.get(i)) + for { + offset <- Seq(0, 1, 63, 64, 65) + } { + val copyBitset = new BitSet(100) + copyBitset.orWithOffset(bitset, offset) + for (i <- offset until 100) { + if (setBits.contains(i - offset)) { + assert(copyBitset.get(i)) + } else { + assert(!copyBitset.get(i)) + } } } } From 2db78eec00b03a5bc5b4f6b0457d93502ee8c1f4 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 10 Oct 2015 23:36:53 +0100 Subject: [PATCH 33/34] Updates test names --- .../org/apache/spark/util/collection/BitSetSuite.scala | 2 +- .../org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 3d7f76f6771f..ab7f7a3a6881 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite class BitSetSuite extends SparkFunSuite { - test("set and get") { + test("basic set and get") { val setBits = Seq(0, 9, 1, 10, 90, 96) val bitset = new BitSet(100) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala index 1e2fe6bbd4bf..0a8f838c0a31 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BitSubvectorSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext */ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { - test("basic ops") { + test("basic set and get") { val from = 1 val to = 4 val bs = new BitSubvector(from, to) @@ -52,7 +52,7 @@ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(copyBs.iterator.toSet === setVals.toSet) } - test("BitSubvector merge") { + test("merge") { val b1 = new BitSubvector(0, 5) b1.set(1) val b2 = new BitSubvector(5, 7) @@ -76,7 +76,7 @@ class BitSubvectorSuite extends SparkFunSuite with MLlibTestSparkContext { } } - test("BitSubvector merge with empty BitSubvectors") { + test("merge with empty BitSubvectors") { val parts = BitSubvector.merge(Array.empty[BitSubvector], Array.empty[BitSubvector]) } } From 7072f91e9fd81f6cefa78775693fee5198f3c2f7 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sat, 14 Nov 2015 20:54:52 +0000 Subject: [PATCH 34/34] Adds tests for 0 until offset --- .../scala/org/apache/spark/util/collection/BitSetSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index ab7f7a3a6881..88e4a5cfcf52 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -51,6 +51,9 @@ class BitSetSuite extends SparkFunSuite { } { val copyBitset = new BitSet(100) copyBitset.orWithOffset(bitset, offset) + for (i <- 0 until offset) { + assert(!copyBitset.get(i)) + } for (i <- offset until 100) { if (setBits.contains(i - offset)) { assert(copyBitset.get(i))