From 711356b422c66e2a80377a9f43fce97282460520 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 8 Sep 2014 09:47:13 -0700 Subject: [PATCH 01/76] [SPARK-3086] [SPARK-3043] [SPARK-3156] [mllib] DecisionTree aggregation improvements MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Summary: 1. Variable numBins for each feature [SPARK-3043] 2. Reduced data reshaping in aggregation [SPARK-3043] 3. Choose ordering for ordered categorical features adaptively [SPARK-3156] 4. Changed nodes to use 1-indexing [SPARK-3086] 5. Small clean-ups Note: This PR looks bigger than it is since I moved several functions from inside findBestSplitsPerGroup to outside of it (to make it clear what was being serialized in the aggregation). Speedups: This update helps most when many features use few bins but a few features use many bins. Some example results on speedups with 2M examples, 3.5K features (15-worker EC2 cluster): * Example where old code was reasonably efficient (1/2 continuous, 1/4 binary, 1/4 20-category): 164.813 --> 116.491 sec * Example where old code wasted many bins (1/10 continuous, 81/100 binary, 9/100 20-category): 128.701 --> 39.334 sec Details: (1) Variable numBins for each feature [SPARK-3043] DecisionTreeMetadata now computes a variable numBins for each feature. It also tracks numSplits. (2) Reduced data reshaping in aggregation [SPARK-3043] Added DTStatsAggregator, a wrapper around the aggregate statistics array for easy but efficient indexing. * Added ImpurityAggregator and ImpurityCalculator classes, to make DecisionTree code more oblivious to the type of impurity. * Design note: I originally tried creating Impurity classes which stored data and storing the aggregates in an Array[Array[Array[Impurity]]]. However, this led to significant slowdowns, perhaps because of overhead in creating so many objects. The aggregate statistics are never reshaped, and cumulative sums are computed in-place. Updated the layout of aggregation functions. The update simplifies things by (1) dividing features into ordered/unordered (instead of ordered/unordered/continuous) and (2) making use of the DTStatsAggregator for indexing. For this update, the following functions were refactored: * updateBinForOrderedFeature * updateBinForUnorderedFeature * binaryOrNotCategoricalBinSeqOp * multiclassWithCategoricalBinSeqOp * regressionBinSeqOp The above 5 functions were replaced with: * orderedBinSeqOp * someUnorderedBinSeqOp Other changes: * calculateGainForSplit now treats all feature types the same way. * Eliminated extractLeftRightNodeAggregates. (3) Choose ordering for ordered categorical features adaptively [SPARK-3156] Updated binsToBestSplit(): * This now computes cumulative sums of stats for ordered features. * For ordered categorical features, it chooses an ordering for categories. (This uses to be done by findSplitsBins.) * Uses iterators to shorten code and avoid building an Array[Array[InformationGainStats]]. Side effects: * In findSplitsBins: A sample of the data is only taken for data with continuous features. It is not needed for data with only categorical features. * In findSplitsBins: splits and bins are no longer pre-computed for ordered categorical features since they are not needed. * TreePoint binning is simpler for categorical features. (4) Changed nodes to use 1-indexing [SPARK-3086] Nodes used to be indexed from 0. Now they are indexed from 1. Node indexing functions are now collected in object Node (Node.scala). (5) Small clean-ups Eliminated functions extractNodeInfo() and extractInfoForLowerLevels() to reduce duplicate code. Eliminated InvalidBinIndex since it is no longer used. CC: mengxr manishamde Please let me know if you have thoughts on this—thanks! Author: Joseph K. Bradley Closes #2125 from jkbradley/dt-opt3alt and squashes the following commits: 42c192a [Joseph K. Bradley] Merge branch 'rfs' into dt-opt3alt d3cc46b [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 00e4404 [Joseph K. Bradley] optimization for TreePoint construction (pre-computing featureArity and isUnordered as arrays) 425716c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs a2acea5 [Joseph K. Bradley] Small optimizations based on profiling aa4e4df [Joseph K. Bradley] Updated DTStatsAggregator with bug fix (nodeString should not be multiplied by statsSize) 4651154 [Joseph K. Bradley] Changed numBins semantics for unordered features. * Before: numBins = numSplits = (1 << k - 1) - 1 * Now: numBins = 2 * numSplits = 2 * [(1 << k - 1) - 1] * This also involved changing the semantics of: ** DecisionTreeMetadata.numUnorderedBins() 1e3b1c7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 1485fcc [Joseph K. Bradley] Made some DecisionTree methods private. 92f934f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt e676da1 [Joseph K. Bradley] Updated documentation for DecisionTree 37ca845 [Joseph K. Bradley] Fixed problem with how DecisionTree handles ordered categorical features. 105f8ab [Joseph K. Bradley] Removed commented-out getEmptyBinAggregates from DecisionTree 062c31d [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 6d32ccd [Joseph K. Bradley] In DecisionTree.binsToBestSplit, changed loops to iterators to shorten code. 807cd00 [Joseph K. Bradley] Finished DTStatsAggregator, a wrapper around the aggregate statistics for easy but hopefully efficient indexing. Modified old ImpurityAggregator classes and renamed them ImpurityCalculator; added ImpurityAggregator classes which work with DTStatsAggregator but do not store data. Unit tests all succeed. f2166fd [Joseph K. Bradley] still working on DTStatsAggregator 92f7118 [Joseph K. Bradley] Added partly written DTStatsAggregator fd8df30 [Joseph K. Bradley] Moved some aggregation helpers outside of findBestSplitsPerGroup d7c53ee [Joseph K. Bradley] Added more doc for ImpurityAggregator a40f8f1 [Joseph K. Bradley] Changed nodes to be indexed from 1. Tests work. 95cad7c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 5f94342 [Joseph K. Bradley] Added treeAggregate since not yet merged from master. Moved node indexing functions to Node. 61c4509 [Joseph K. Bradley] Fixed bugs from merge: missing DT timer call, and numBins setting. Cleaned up DT Suite some. 3ba7166 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 b314659 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 9c83363 [Joseph K. Bradley] partial merge but not done yet 45f7ea7 [Joseph K. Bradley] partial merge, not yet done 5fce635 [Joseph K. Bradley] Merge branch 'dt-opt2' into dt-opt3 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 dd4d3aa [Joseph K. Bradley] Mid-process in bug fix: bug for binary classification with categorical features * Bug: Categorical features were all treated as ordered for binary classification. This is possible but would require the bin ordering to be determined on-the-fly after the aggregation. Currently, the ordering is determined a priori and fixed for all splits. * (Temp) Fix: Treat low-arity categorical features as unordered for binary classification. * Related change: I removed most tests for isMulticlass in the code. I instead test metadata for whether there are unordered features. * Status: The bug may be fixed, but more testing needs to be done. 438a660 [Joseph K. Bradley] removed subsampling for mnist8m from DT 86e217f [Joseph K. Bradley] added cache to DT input e3c84cc [Joseph K. Bradley] Added stuff fro mnist8m to D T Runner 51ef781 [Joseph K. Bradley] Fixed bug introduced by last commit: Variance impurity calculation was incorrect since counts were swapped accidentally fd65372 [Joseph K. Bradley] Major changes: * Created ImpurityAggregator classes, rather than old aggregates. * Feature split/bin semantics are based on ordered vs. unordered ** E.g.: numSplits = numBins for all unordered features, and numSplits = numBins - 1 for all ordered features. * numBins can differ for each feature c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals --- .../spark/mllib/tree/DecisionTree.scala | 1341 ++++++----------- .../mllib/tree/impl/DTStatsAggregator.scala | 213 +++ .../tree/impl/DecisionTreeMetadata.scala | 73 +- .../spark/mllib/tree/impl/TreePoint.scala | 93 +- .../spark/mllib/tree/impurity/Entropy.scala | 84 ++ .../spark/mllib/tree/impurity/Gini.scala | 84 ++ .../spark/mllib/tree/impurity/Impurity.scala | 127 ++ .../spark/mllib/tree/impurity/Variance.scala | 72 + .../apache/spark/mllib/tree/model/Bin.scala | 7 +- .../apache/spark/mllib/tree/model/Node.scala | 85 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 391 ++--- 11 files changed, 1322 insertions(+), 1248 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 5cdd258f6c20b..dd766c12d28a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -28,8 +28,9 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impl._ import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} +import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -65,36 +66,41 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val retaggedInput = input.retag(classOf[LabeledPoint]) val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) logDebug("algo = " + strategy.algo) + logDebug("maxBins = " + metadata.maxBins) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. timer.start("findSplitsBins") val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) - val numBins = bins(0).length timer.stop("findSplitsBins") - logDebug("numBins = " + numBins) + logDebug("numBins: feature: number of bins") + logDebug(Range(0, metadata.numFeatures).map { featureIndex => + s"\t$featureIndex\t${metadata.numBins(featureIndex)}" + }.mkString("\n")) // Bin feature values (TreePoint representation). // Cache input RDD for speedup during multiple passes. val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) .persist(StorageLevel.MEMORY_AND_DISK) - val numFeatures = metadata.numFeatures // depth of the decision tree val maxDepth = strategy.maxDepth - // the max number of nodes possible given the depth of the tree - val maxNumNodes = (2 << maxDepth) - 1 + require(maxDepth <= 30, + s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") + // Number of nodes to allocate: max number of nodes possible given the depth of the tree, plus 1 + val maxNumNodesPlus1 = Node.startIndexInLevel(maxDepth + 1) // Initialize an array to hold parent impurity calculations for each node. - val parentImpurities = new Array[Double](maxNumNodes) + val parentImpurities = new Array[Double](maxNumNodesPlus1) // dummy value for top node (updated during first split calculation) - val nodes = new Array[Node](maxNumNodes) + val nodes = new Array[Node](maxNumNodesPlus1) // Calculate level for single group construction // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = DecisionTree.getElementsPerNode(metadata, numBins) + // TODO: Calculate memory usage more precisely. + val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -124,26 +130,29 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, - metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) + val splitsStatsForLevel: Array[(Split, InformationGainStats)] = + DecisionTree.findBestSplits(treeInput, parentImpurities, + metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") - val levelNodeIndexOffset = (1 << level) - 1 + val levelNodeIndexOffset = Node.startIndexInLevel(level) for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { val nodeIndex = levelNodeIndexOffset + index - val isLeftChild = level != 0 && nodeIndex % 2 == 1 - val parentNodeIndex = if (isLeftChild) { // -1 for root node - (nodeIndex - 1) / 2 - } else { - (nodeIndex - 2) / 2 - } + // Extract info for this node (index) at the current level. timer.start("extractNodeInfo") - extractNodeInfo(nodeSplitStats, level, index, nodes) + val split = nodeSplitStats._1 + val stats = nodeSplitStats._2 + val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) + val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) + logDebug("Node = " + node) + nodes(nodeIndex) = node timer.stop("extractNodeInfo") + if (level != 0) { // Set parent. - if (isLeftChild) { + val parentNodeIndex = Node.parentIndex(nodeIndex) + if (Node.isLeftChild(nodeIndex)) { nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) } else { nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) @@ -151,11 +160,21 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo } // Extract info for nodes at the next lower level. timer.start("extractInfoForLowerLevels") - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities) + if (level < maxDepth) { + val leftChildIndex = Node.leftChildIndex(nodeIndex) + val leftImpurity = stats.leftImpurity + logDebug("leftChildIndex = " + leftChildIndex + ", impurity = " + leftImpurity) + parentImpurities(leftChildIndex) = leftImpurity + + val rightChildIndex = Node.rightChildIndex(nodeIndex) + val rightImpurity = stats.rightImpurity + logDebug("rightChildIndex = " + rightChildIndex + ", impurity = " + rightImpurity) + parentImpurities(rightChildIndex) = rightImpurity + } timer.stop("extractInfoForLowerLevels") - logDebug("final best split = " + nodeSplitStats._1) + logDebug("final best split = " + split) } - require((1 << level) == splitsStatsForLevel.length) + require(Node.maxNodesInLevel(level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -171,7 +190,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("#####################################") // Initialize the top or root node of the tree. - val topNode = nodes(0) + val topNode = nodes(1) // Build the full tree using the node info calculated in the level-wise best split calculations. topNode.build(nodes) @@ -183,47 +202,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo new DecisionTreeModel(topNode, strategy.algo) } - /** - * Extract the decision tree node information for the given tree level and node index - */ - private def extractNodeInfo( - nodeSplitStats: (Split, InformationGainStats), - level: Int, - index: Int, - nodes: Array[Node]): Unit = { - val split = nodeSplitStats._1 - val stats = nodeSplitStats._2 - val nodeIndex = (1 << level) - 1 + index - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - nodes(nodeIndex) = node - } - - /** - * Extract the decision tree node information for the children of the node - */ - private def extractInfoForLowerLevels( - level: Int, - index: Int, - maxDepth: Int, - nodeSplitStats: (Split, InformationGainStats), - parentImpurities: Array[Double]): Unit = { - - if (level >= maxDepth) { - return - } - - val leftNodeIndex = (2 << level) - 1 + 2 * index - val leftImpurity = nodeSplitStats._2.leftImpurity - logDebug("leftNodeIndex = " + leftNodeIndex + ", impurity = " + leftImpurity) - parentImpurities(leftNodeIndex) = leftImpurity - - val rightNodeIndex = leftNodeIndex + 1 - val rightImpurity = nodeSplitStats._2.rightImpurity - logDebug("rightNodeIndex = " + rightNodeIndex + ", impurity = " + rightImpurity) - parentImpurities(rightNodeIndex) = rightImpurity - } } object DecisionTree extends Serializable with Logging { @@ -425,9 +403,6 @@ object DecisionTree extends Serializable with Logging { impurity, maxDepth, maxBins) } - - private val InvalidBinIndex = -1 - /** * Returns an array of optimal splits for all nodes at a given level. Splits the task into * multiple groups if the level-wise training task could lead to memory overflow. @@ -436,12 +411,12 @@ object DecisionTree extends Serializable with Logging { * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param splits possible splits for all features - * @param bins possible bins for all features + * @param splits possible splits for all features, indexed (numFeatures)(numSplits) + * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. * @return array (over nodes) of splits with best split for each node at a given level. */ - protected[tree] def findBestSplits( + private[tree] def findBestSplits( input: RDD[TreePoint], parentImpurities: Array[Double], metadata: DecisionTreeMetadata, @@ -474,6 +449,138 @@ object DecisionTree extends Serializable with Logging { } } + /** + * Get the node index corresponding to this data point. + * This function mimics prediction, passing an example from the root node down to a node + * at the current level being trained; that node's index is returned. + * + * @param node Node in tree from which to classify the given data point. + * @param binnedFeatures Binned feature vector for data point. + * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param unorderedFeatures Set of indices of unordered features. + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. + * Note: This is the global node index, i.e., the index used in the tree. + * This index is different from the index used during training a particular + * set of nodes in a (level, group). + */ + private def predictNodeIndex( + node: Node, + binnedFeatures: Array[Int], + bins: Array[Array[Bin]], + unorderedFeatures: Set[Int]): Int = { + if (node.isLeaf) { + node.id + } else { + val featureIndex = node.split.get.feature + val splitLeft = node.split.get.featureType match { + case Continuous => { + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] + // We do not need to check lowSplit since bins are separated by splits. + featureValueUpperBound <= node.split.get.threshold + } + case Categorical => { + val featureValue = binnedFeatures(featureIndex) + node.split.get.categories.contains(featureValue) + } + case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") + } + if (node.leftNode.isEmpty || node.rightNode.isEmpty) { + // Return index from next layer of nodes to train + if (splitLeft) { + Node.leftChildIndex(node.id) + } else { + Node.rightChildIndex(node.id) + } + } else { + if (splitLeft) { + predictNodeIndex(node.leftNode.get, binnedFeatures, bins, unorderedFeatures) + } else { + predictNodeIndex(node.rightNode.get, binnedFeatures, bins, unorderedFeatures) + } + } + } + } + + /** + * Helper for binSeqOp, for data which can contain a mix of ordered and unordered features. + * + * For ordered features, a single bin is updated. + * For unordered features, bins correspond to subsets of categories; either the left or right bin + * for each subset is updated. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param unorderedFeatures Set of indices of unordered features. + */ + private def mixedBinSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint, + nodeIndex: Int, + bins: Array[Array[Bin]], + unorderedFeatures: Set[Int]): Unit = { + // Iterate over all features. + val numFeatures = treePoint.binnedFeatures.size + val nodeOffset = agg.getNodeOffset(nodeIndex) + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (unorderedFeatures.contains(featureIndex)) { + // Unordered feature + val featureValue = treePoint.binnedFeatures(featureIndex) + val (leftNodeFeatureOffset, rightNodeFeatureOffset) = + agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + // Update the left or right bin for each split. + val numSplits = agg.numSplits(featureIndex) + var splitIndex = 0 + while (splitIndex < numSplits) { + if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { + agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label) + } else { + agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label) + } + splitIndex += 1 + } + } else { + // Ordered feature + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, featureIndex, binIndex, treePoint.label) + } + featureIndex += 1 + } + } + + /** + * Helper for binSeqOp, for regression and for classification with only ordered features. + * + * For each feature, the sufficient statistics of one bin are updated. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @return agg + */ + private def orderedBinSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint, + nodeIndex: Int): Unit = { + val label = treePoint.label + val nodeOffset = agg.getNodeOffset(nodeIndex) + // Iterate over all features. + val numFeatures = agg.numFeatures + var featureIndex = 0 + while (featureIndex < numFeatures) { + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, featureIndex, binIndex, label) + featureIndex += 1 + } + } + /** * Returns an array of optimal splits for a group of nodes at a given level * @@ -481,8 +588,9 @@ object DecisionTree extends Serializable with Logging { * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param splits possible splits for all features - * @param bins possible bins for all features, indexed as (numFeatures)(numBins) + * @param nodes Array of all nodes in the tree. Used for matching data points to nodes. + * @param splits possible splits for all features, indexed (numFeatures)(numSplits) + * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. * @return array of splits with best splits for all nodes at a given level. @@ -527,88 +635,22 @@ object DecisionTree extends Serializable with Logging { // numNodes: Number of nodes in this (level of tree, group), // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = (1 << level) / numGroups + val numNodes = Node.maxNodesInLevel(level) / numGroups logDebug("numNodes = " + numNodes) - // Find the number of features by looking at the first sample. - val numFeatures = metadata.numFeatures - logDebug("numFeatures = " + numFeatures) - - // numBins: Number of bins = 1 + number of possible splits - val numBins = bins(0).length - logDebug("numBins = " + numBins) - - val numClasses = metadata.numClasses - logDebug("numClasses = " + numClasses) - - val isMulticlass = metadata.isMulticlass - logDebug("isMulticlass = " + isMulticlass) - - val isMulticlassWithCategoricalFeatures = metadata.isMulticlassWithCategoricalFeatures - logDebug("isMultiClassWithCategoricalFeatures = " + isMulticlassWithCategoricalFeatures) + logDebug("numFeatures = " + metadata.numFeatures) + logDebug("numClasses = " + metadata.numClasses) + logDebug("isMulticlass = " + metadata.isMulticlass) + logDebug("isMulticlassWithCategoricalFeatures = " + + metadata.isMulticlassWithCategoricalFeatures) // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex - /** - * Get the node index corresponding to this data point. - * This function mimics prediction, passing an example from the root node down to a node - * at the current level being trained; that node's index is returned. - * - * @return Leaf index if the data point reaches a leaf. - * Otherwise, last node reachable in tree matching this example. - */ - def predictNodeIndex(node: Node, binnedFeatures: Array[Int]): Int = { - if (node.isLeaf) { - node.id - } else { - val featureIndex = node.split.get.feature - val splitLeft = node.split.get.featureType match { - case Continuous => { - val binIndex = binnedFeatures(featureIndex) - val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold - // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] - // We do not need to check lowSplit since bins are separated by splits. - featureValueUpperBound <= node.split.get.threshold - } - case Categorical => { - val featureValue = if (metadata.isUnordered(featureIndex)) { - binnedFeatures(featureIndex) - } else { - val binIndex = binnedFeatures(featureIndex) - bins(featureIndex)(binIndex).category - } - node.split.get.categories.contains(featureValue) - } - case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") - } - if (node.leftNode.isEmpty || node.rightNode.isEmpty) { - // Return index from next layer of nodes to train - if (splitLeft) { - node.id * 2 + 1 // left - } else { - node.id * 2 + 2 // right - } - } else { - if (splitLeft) { - predictNodeIndex(node.leftNode.get, binnedFeatures) - } else { - predictNodeIndex(node.rightNode.get, binnedFeatures) - } - } - } - } - - def nodeIndexToLevel(idx: Int): Int = { - if (idx == 0) { - 0 - } else { - math.floor(math.log(idx) / math.log(2)).toInt - } - } - - // Used for treePointToNodeIndex - val levelOffset = (1 << level) - 1 + // Used for treePointToNodeIndex to get an index for this (level, group). + // - Node.startIndexInLevel(level) gives the global index offset for nodes at this level. + // - groupShift corrects for groups in this level before the current group. + val globalNodeIndexOffset = Node.startIndexInLevel(level) + groupShift /** * Find the node index for the given example. @@ -619,661 +661,254 @@ object DecisionTree extends Serializable with Logging { if (level == 0) { 0 } else { - val globalNodeIndex = predictNodeIndex(nodes(0), treePoint.binnedFeatures) - // Get index for this (level, group). - globalNodeIndex - levelOffset - groupShift - } - } - - /** - * Increment aggregate in location for (node, feature, bin, label). - * - * @param treePoint Data point being aggregated. - * @param agg Array storing aggregate calculation, of size: - * numClasses * numBins * numFeatures * numNodes. - * Indexed by (node, feature, bin, label) where label is the least significant bit. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def updateBinForOrderedFeature( - treePoint: TreePoint, - agg: Array[Double], - nodeIndex: Int, - featureIndex: Int): Unit = { - // Update the left or right count for one bin. - val aggIndex = - numClasses * numBins * numFeatures * nodeIndex + - numClasses * numBins * featureIndex + - numClasses * treePoint.binnedFeatures(featureIndex) + - treePoint.label.toInt - agg(aggIndex) += 1 - } - - /** - * Increment aggregate in location for (nodeIndex, featureIndex, [bins], label), - * where [bins] ranges over all bins. - * Updates left or right side of aggregate depending on split. - * - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @param treePoint Data point being aggregated. - * @param agg Indexed by (left/right, node, feature, bin, label) - * where label is the least significant bit. - * The left/right specifier is a 0/1 index indicating left/right child info. - * @param rightChildShift Offset for right side of agg. - */ - def updateBinForUnorderedFeature( - nodeIndex: Int, - featureIndex: Int, - treePoint: TreePoint, - agg: Array[Double], - rightChildShift: Int): Unit = { - val featureValue = treePoint.binnedFeatures(featureIndex) - // Update the left or right count for one bin. - val aggShift = - numClasses * numBins * numFeatures * nodeIndex + - numClasses * numBins * featureIndex + - treePoint.label.toInt - // Find all matching bins and increment their values - val featureCategories = metadata.featureArity(featureIndex) - val numCategoricalBins = (1 << featureCategories - 1) - 1 - var binIndex = 0 - while (binIndex < numCategoricalBins) { - val aggIndex = aggShift + binIndex * numClasses - if (bins(featureIndex)(binIndex).highSplit.categories.contains(featureValue)) { - agg(aggIndex) += 1 - } else { - agg(rightChildShift + aggIndex) += 1 - } - binIndex += 1 - } - } - - /** - * Helper for binSeqOp. - * - * @param agg Array storing aggregate calculation, of size: - * numClasses * numBins * numFeatures * numNodes. - * Indexed by (node, feature, bin, label) where label is the least significant bit. - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def binaryOrNotCategoricalBinSeqOp( - agg: Array[Double], - treePoint: TreePoint, - nodeIndex: Int): Unit = { - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) - featureIndex += 1 - } - } - - val rightChildShift = numClasses * numBins * numFeatures * numNodes - - /** - * Helper for binSeqOp. - * - * @param agg Array storing aggregate calculation. - * For ordered features, this is of size: - * numClasses * numBins * numFeatures * numNodes. - * For unordered features, this is of size: - * 2 * numClasses * numBins * numFeatures * numNodes. - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def multiclassWithCategoricalBinSeqOp( - agg: Array[Double], - treePoint: TreePoint, - nodeIndex: Int): Unit = { - val label = treePoint.label - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isUnordered(featureIndex)) { - updateBinForUnorderedFeature(nodeIndex, featureIndex, treePoint, agg, rightChildShift) - } else { - updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) - } - featureIndex += 1 - } - } - - /** - * Performs a sequential aggregation over a partition for regression. - * For l nodes, k features, - * the count, sum, sum of squares of one of the p bins is incremented. - * - * @param agg Array storing aggregate calculation, updated by this function. - * Size: 3 * numBins * numFeatures * numNodes - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @return agg - */ - def regressionBinSeqOp(agg: Array[Double], treePoint: TreePoint, nodeIndex: Int): Unit = { - val label = treePoint.label - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Update count, sum, and sum^2 for one bin. - val binIndex = treePoint.binnedFeatures(featureIndex) - val aggIndex = - 3 * numBins * numFeatures * nodeIndex + - 3 * numBins * featureIndex + - 3 * binIndex - agg(aggIndex) += 1 - agg(aggIndex + 1) += label - agg(aggIndex + 2) += label * label - featureIndex += 1 + val globalNodeIndex = + predictNodeIndex(nodes(1), treePoint.binnedFeatures, bins, metadata.unorderedFeatures) + globalNodeIndex - globalNodeIndexOffset } } /** * Performs a sequential aggregation over a partition. - * For l nodes, k features, - * For classification: - * Either the left count or the right count of one of the bins is - * incremented based upon whether the feature is classified as 0 or 1. - * For regression: - * The count, sum, sum of squares of one of the bins is incremented. * - * @param agg Array storing aggregate calculation, updated by this function. - * Size for classification: - * numClasses * numBins * numFeatures * numNodes for ordered features, or - * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. - * Size for regression: - * 3 * numBins * numFeatures * numNodes. + * Each data point contributes to one node. For each feature, + * the aggregate sufficient statistics are updated for the relevant bins. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). * @param treePoint Data point being aggregated. * @return agg */ - def binSeqOp(agg: Array[Double], treePoint: TreePoint): Array[Double] = { + def binSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint): DTStatsAggregator = { val nodeIndex = treePointToNodeIndex(treePoint) // If the example does not reach this level, then nodeIndex < 0. // If the example reaches this level but is handled in a different group, // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). if (nodeIndex >= 0 && nodeIndex < numNodes) { - if (metadata.isClassification) { - if (isMulticlassWithCategoricalFeatures) { - multiclassWithCategoricalBinSeqOp(agg, treePoint, nodeIndex) - } else { - binaryOrNotCategoricalBinSeqOp(agg, treePoint, nodeIndex) - } + if (metadata.unorderedFeatures.isEmpty) { + orderedBinSeqOp(agg, treePoint, nodeIndex) } else { - regressionBinSeqOp(agg, treePoint, nodeIndex) + mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures) } } agg } - // Calculate bin aggregate length for classification or regression. - val binAggregateLength = numNodes * getElementsPerNode(metadata, numBins) - logDebug("binAggregateLength = " + binAggregateLength) - - /** - * Combines the aggregates from partitions. - * @param agg1 Array containing aggregates from one or more partitions - * @param agg2 Array containing aggregates from one or more partitions - * @return Combined aggregate from agg1 and agg2 - */ - def binCombOp(agg1: Array[Double], agg2: Array[Double]): Array[Double] = { - var index = 0 - val combinedAggregate = new Array[Double](binAggregateLength) - while (index < binAggregateLength) { - combinedAggregate(index) = agg1(index) + agg2(index) - index += 1 - } - combinedAggregate - } - // Calculate bin aggregates. timer.start("aggregation") - val binAggregates = { - input.treeAggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) + val binAggregates: DTStatsAggregator = { + val initAgg = new DTStatsAggregator(metadata, numNodes) + input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) } timer.stop("aggregation") - logDebug("binAggregates.length = " + binAggregates.length) - /** - * Calculate the information gain for a given (feature, split) based upon left/right aggregates. - * @param leftNodeAgg left node aggregates for this (feature, split) - * @param rightNodeAgg right node aggregate for this (feature, split) - * @param topImpurity impurity of the parent node - * @return information gain and statistics for all splits - */ - def calculateGainForSplit( - leftNodeAgg: Array[Double], - rightNodeAgg: Array[Double], - topImpurity: Double): InformationGainStats = { - if (metadata.isClassification) { - val leftTotalCount = leftNodeAgg.sum - val rightTotalCount = rightNodeAgg.sum - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val rootNodeCounts = new Array[Double](numClasses) - var classIndex = 0 - while (classIndex < numClasses) { - rootNodeCounts(classIndex) = leftNodeAgg(classIndex) + rightNodeAgg(classIndex) - classIndex += 1 - } - metadata.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) - } - } - - val totalCount = leftTotalCount + rightTotalCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) - } - - // Sum of count for each label - val leftrightNodeAgg: Array[Double] = - leftNodeAgg.zip(rightNodeAgg).map { case (leftCount, rightCount) => - leftCount + rightCount - } - - def indexOfLargestArrayElement(array: Array[Double]): Int = { - val result = array.foldLeft(-1, Double.MinValue, 0) { - case ((maxIndex, maxValue, currentIndex), currentValue) => - if (currentValue > maxValue) { - (currentIndex, currentValue, currentIndex + 1) - } else { - (maxIndex, maxValue, currentIndex + 1) - } - } - if (result._1 < 0) { - throw new RuntimeException("DecisionTree internal error:" + - " calculateGainForSplit failed in indexOfLargestArrayElement") - } - result._1 - } - - val predict = indexOfLargestArrayElement(leftrightNodeAgg) - val prob = leftrightNodeAgg(predict) / totalCount - - val leftImpurity = if (leftTotalCount == 0) { - topImpurity - } else { - metadata.impurity.calculate(leftNodeAgg, leftTotalCount) - } - val rightImpurity = if (rightTotalCount == 0) { - topImpurity - } else { - metadata.impurity.calculate(rightNodeAgg, rightTotalCount) - } - - val leftWeight = leftTotalCount / totalCount - val rightWeight = rightTotalCount / totalCount - - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) - - } else { - // Regression - - val leftCount = leftNodeAgg(0) - val leftSum = leftNodeAgg(1) - val leftSumSquares = leftNodeAgg(2) + // Calculate best splits for all nodes at a given level + timer.start("chooseSplits") + val bestSplits = new Array[(Split, InformationGainStats)](numNodes) + // Iterating over all nodes at this level + var nodeIndex = 0 + while (nodeIndex < numNodes) { + val nodeImpurity = parentImpurities(globalNodeIndexOffset + nodeIndex) + logDebug("node impurity = " + nodeImpurity) + bestSplits(nodeIndex) = + binsToBestSplit(binAggregates, nodeIndex, nodeImpurity, level, metadata, splits) + logDebug("best split = " + bestSplits(nodeIndex)._1) + nodeIndex += 1 + } + timer.stop("chooseSplits") - val rightCount = rightNodeAgg(0) - val rightSum = rightNodeAgg(1) - val rightSumSquares = rightNodeAgg(2) + bestSplits + } - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val count = leftCount + rightCount - val sum = leftSum + rightSum - val sumSquares = leftSumSquares + rightSumSquares - metadata.impurity.calculate(count, sum, sumSquares) - } - } + /** + * Calculate the information gain for a given (feature, split) based upon left/right aggregates. + * @param leftImpurityCalculator left node aggregates for this (feature, split) + * @param rightImpurityCalculator right node aggregate for this (feature, split) + * @param topImpurity impurity of the parent node + * @return information gain and statistics for all splits + */ + private def calculateGainForSplit( + leftImpurityCalculator: ImpurityCalculator, + rightImpurityCalculator: ImpurityCalculator, + topImpurity: Double, + level: Int, + metadata: DecisionTreeMetadata): InformationGainStats = { - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, - rightSum / rightCount) - } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity, topImpurity, - Double.MinValue, leftSum / leftCount) - } + val leftCount = leftImpurityCalculator.count + val rightCount = rightImpurityCalculator.count - val leftImpurity = metadata.impurity.calculate(leftCount, leftSum, leftSumSquares) - val rightImpurity = metadata.impurity.calculate(rightCount, rightSum, rightSumSquares) + val totalCount = leftCount + rightCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + // impurity of parent node + val impurity = if (level > 0) { + topImpurity + } else { + parentNodeAgg.calculate() + } - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + val predict = parentNodeAgg.predict + val prob = parentNodeAgg.prob(predict) - val predict = (leftSum + rightSum) / (leftCount + rightCount) - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) - } - } + val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 + val rightImpurity = rightImpurityCalculator.calculate() - /** - * Extracts left and right split aggregates. - * @param binData Aggregate array slice from getBinDataForNode. - * For classification: - * For unordered features, this is leftChildData ++ rightChildData, - * each of which is indexed by (feature, split/bin, class), - * with class being the least significant bit. - * For ordered features, this is of size numClasses * numBins * numFeatures. - * For regression: - * This is of size 2 * numFeatures * numBins. - * @return (leftNodeAgg, rightNodeAgg) pair of arrays. - * For classification, each array is of size (numFeatures, (numBins - 1), numClasses). - * For regression, each array is of size (numFeatures, (numBins - 1), 3). - * - */ - def extractLeftRightNodeAggregates( - binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { - - - /** - * The input binData is indexed as (feature, bin, class). - * This computes cumulative sums over splits. - * Each (feature, class) pair is handled separately. - * Note: numSplits = numBins - 1. - * @param leftNodeAgg Each (feature, class) slice is an array over splits. - * Element i (i = 0, ..., numSplits - 2) is set to be - * the cumulative sum (from left) over binData for bins 0, ..., i. - * @param rightNodeAgg Each (feature, class) slice is an array over splits. - * Element i (i = 1, ..., numSplits - 1) is set to be - * the cumulative sum (from right) over binData for bins - * numBins - 1, ..., numBins - 1 - i. - */ - def findAggForOrderedFeatureClassification( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - // shift for this featureIndex - val shift = numClasses * featureIndex * numBins - - var classIndex = 0 - while (classIndex < numClasses) { - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0)(classIndex) = binData(shift + classIndex) - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(numBins - 2)(classIndex) - = binData(shift + (numClasses * (numBins - 1)) + classIndex) - classIndex += 1 - } + val leftWeight = leftCount / totalCount.toDouble + val rightWeight = rightCount / totalCount.toDouble - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - var innerClassIndex = 0 - while (innerClassIndex < numClasses) { - leftNodeAgg(featureIndex)(splitIndex)(innerClassIndex) - = binData(shift + numClasses * splitIndex + innerClassIndex) + - leftNodeAgg(featureIndex)(splitIndex - 1)(innerClassIndex) - rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(innerClassIndex) = - binData(shift + (numClasses * (numBins - 1 - splitIndex) + innerClassIndex)) + - rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(innerClassIndex) - innerClassIndex += 1 - } - splitIndex += 1 - } - } + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - /** - * Reshape binData for this feature. - * Indexes binData as (feature, split, class) with class as the least significant bit. - * @param leftNodeAgg leftNodeAgg(featureIndex)(splitIndex)(classIndex) = aggregate value - */ - def findAggForUnorderedFeatureClassification( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - val rightChildShift = numClasses * numBins * numFeatures - var splitIndex = 0 - while (splitIndex < numBins - 1) { - var classIndex = 0 - while (classIndex < numClasses) { - // shift for this featureIndex - val shift = numClasses * featureIndex * numBins + splitIndex * numClasses - val leftBinValue = binData(shift + classIndex) - val rightBinValue = binData(rightChildShift + shift + classIndex) - leftNodeAgg(featureIndex)(splitIndex)(classIndex) = leftBinValue - rightNodeAgg(featureIndex)(splitIndex)(classIndex) = rightBinValue - classIndex += 1 - } - splitIndex += 1 - } - } + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + } - def findAggForRegression( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - // shift for this featureIndex - val shift = 3 * featureIndex * numBins - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(0)(1) = binData(shift + 1) - leftNodeAgg(featureIndex)(0)(2) = binData(shift + 2) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(numBins - 2)(0) = - binData(shift + (3 * (numBins - 1))) - rightNodeAgg(featureIndex)(numBins - 2)(1) = - binData(shift + (3 * (numBins - 1)) + 1) - rightNodeAgg(featureIndex)(numBins - 2)(2) = - binData(shift + (3 * (numBins - 1)) + 2) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - var i = 0 // index for regression histograms - while (i < 3) { // count, sum, sum^2 - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(splitIndex)(i) = binData(shift + 3 * splitIndex + i) + - leftNodeAgg(featureIndex)(splitIndex - 1)(i) - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(i) = - binData(shift + (3 * (numBins - 1 - splitIndex) + i)) + - rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(i) - i += 1 - } - splitIndex += 1 - } - } + /** + * Find the best split for a node. + * @param binAggregates Bin statistics. + * @param nodeIndex Index for node to split in this (level, group). + * @param nodeImpurity Impurity of the node (nodeIndex). + * @return tuple for best split: (Split, information gain) + */ + private def binsToBestSplit( + binAggregates: DTStatsAggregator, + nodeIndex: Int, + nodeImpurity: Double, + level: Int, + metadata: DecisionTreeMetadata, + splits: Array[Array[Split]]): (Split, InformationGainStats) = { - if (metadata.isClassification) { - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isUnordered(featureIndex)) { - findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - featureIndex += 1 - } - (leftNodeAgg, rightNodeAgg) - } else { - // Regression - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) - featureIndex += 1 - } - (leftNodeAgg, rightNodeAgg) - } - } + logDebug("node impurity = " + nodeImpurity) - /** - * Calculates information gain for all nodes splits. - */ - def calculateGainsForAllNodeSplits( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - nodeImpurity: Double): Array[Array[InformationGainStats]] = { - val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) - - var featureIndex = 0 - while (featureIndex < numFeatures) { - val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + // For each (feature, split), calculate the gain, and select the best (feature, split). + Range(0, metadata.numFeatures).map { featureIndex => + val numSplits = metadata.numSplits(featureIndex) + if (metadata.isContinuous(featureIndex)) { + // Cumulative sum (scanLeft) of bin statistics. + // Afterwards, binAggregates for a bin is the sum of aggregates for + // that bin + all preceding bins. + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) var splitIndex = 0 - while (splitIndex < numSplitsForFeature) { - gains(featureIndex)(splitIndex) = - calculateGainForSplit(leftNodeAgg(featureIndex)(splitIndex), - rightNodeAgg(featureIndex)(splitIndex), nodeImpurity) + while (splitIndex < numSplits) { + binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) splitIndex += 1 } - featureIndex += 1 - } - gains - } - - /** - * Get the number of splits for a feature. - */ - def getNumSplitsForFeature(featureIndex: Int): Int = { - if (metadata.isContinuous(featureIndex)) { - numBins - 1 + // Find best split. + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { case splitIdx => + val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) + val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) + rightChildStats.subtract(leftChildStats) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIdx, gainStats) + }.maxBy(_._2.gain) + (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) + } else if (metadata.isUnordered(featureIndex)) { + // Unordered categorical feature + val (leftChildOffset, rightChildOffset) = + binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { splitIndex => + val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) + val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIndex, gainStats) + }.maxBy(_._2.gain) + (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { - // Categorical feature - val featureCategories = metadata.featureArity(featureIndex) - if (metadata.isUnordered(featureIndex)) { - (1 << featureCategories - 1) - 1 - } else { - featureCategories - } - } - } - - /** - * Find the best split for a node. - * @param binData Bin data slice for this node, given by getBinDataForNode. - * @param nodeImpurity impurity of the top node - * @return tuple of split and information gain - */ - def binsToBestSplit( - binData: Array[Double], - nodeImpurity: Double): (Split, InformationGainStats) = { - - logDebug("node impurity = " + nodeImpurity) - - // Extract left right node aggregates. - val (leftNodeAgg, rightNodeAgg) = extractLeftRightNodeAggregates(binData) - - // Calculate gains for all splits. - val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity) - - val (bestFeatureIndex, bestSplitIndex, gainStats) = { - // Initialize with infeasible values. - var bestFeatureIndex = Int.MinValue - var bestSplitIndex = Int.MinValue - var bestGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, -1.0) - // Iterate over features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Iterate over all splits. - var splitIndex = 0 - val numSplitsForFeature = getNumSplitsForFeature(featureIndex) - while (splitIndex < numSplitsForFeature) { - val gainStats = gains(featureIndex)(splitIndex) - if (gainStats.gain > bestGainStats.gain) { - bestGainStats = gainStats - bestFeatureIndex = featureIndex - bestSplitIndex = splitIndex + // Ordered categorical feature + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) + val numBins = metadata.numBins(featureIndex) + + /* Each bin is one category (feature value). + * The bins are ordered based on centroidForCategories, and this ordering determines which + * splits are considered. (With K categories, we consider K - 1 possible splits.) + * + * centroidForCategories is a list: (category, centroid) + */ + val centroidForCategories = if (metadata.isMulticlass) { + // For categorical variables in multiclass classification, + // the bins are ordered by the impurity of their corresponding labels. + Range(0, numBins).map { case featureValue => + val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + categoryStats.calculate() + } else { + Double.MaxValue } - splitIndex += 1 + (featureValue, centroid) + } + } else { // regression or binary classification + // For categorical variables in regression and binary classification, + // the bins are ordered by the centroid of their corresponding labels. + Range(0, numBins).map { case featureValue => + val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + categoryStats.predict + } else { + Double.MaxValue + } + (featureValue, centroid) } - featureIndex += 1 } - (bestFeatureIndex, bestSplitIndex, bestGainStats) - } - logDebug("best split = " + splits(bestFeatureIndex)(bestSplitIndex)) - logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex)) + logDebug("Centroids for categorical variable: " + centroidForCategories.mkString(",")) - (splits(bestFeatureIndex)(bestSplitIndex), gainStats) - } + // bins sorted by centroids + val categoriesSortedByCentroid = centroidForCategories.toList.sortBy(_._2) - /** - * Get bin data for one node. - */ - def getBinDataForNode(node: Int): Array[Double] = { - if (metadata.isClassification) { - if (isMulticlassWithCategoricalFeatures) { - val shift = numClasses * node * numBins * numFeatures - val rightChildShift = numClasses * numBins * numFeatures * numNodes - val binsForNode = { - val leftChildData - = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - val rightChildData - = binAggregates.slice(rightChildShift + shift, - rightChildShift + shift + numClasses * numBins * numFeatures) - leftChildData ++ rightChildData - } - binsForNode - } else { - val shift = numClasses * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - binsForNode + logDebug("Sorted centroids for categorical variable = " + + categoriesSortedByCentroid.mkString(",")) + + // Cumulative sum (scanLeft) of bin statistics. + // Afterwards, binAggregates for a bin is the sum of aggregates for + // that bin + all preceding bins. + var splitIndex = 0 + while (splitIndex < numSplits) { + val currentCategory = categoriesSortedByCentroid(splitIndex)._1 + val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 + binAggregates.mergeForNodeFeature(nodeFeatureOffset, nextCategory, currentCategory) + splitIndex += 1 } - } else { - // Regression - val shift = 3 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) - binsForNode + // lastCategory = index of bin with total aggregates for this (node, feature) + val lastCategory = categoriesSortedByCentroid.last._1 + // Find best split. + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { splitIndex => + val featureValue = categoriesSortedByCentroid(splitIndex)._1 + val leftChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val rightChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) + rightChildStats.subtract(leftChildStats) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIndex, gainStats) + }.maxBy(_._2.gain) + val categoriesForSplit = + categoriesSortedByCentroid.map(_._1.toDouble).slice(0, bestFeatureSplitIndex + 1) + val bestFeatureSplit = + new Split(featureIndex, Double.MinValue, Categorical, categoriesForSplit) + (bestFeatureSplit, bestFeatureGainStats) } - } - - // Calculate best splits for all nodes at a given level - timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats)](numNodes) - // Iterating over all nodes at this level - var node = 0 - while (node < numNodes) { - val nodeImpurityIndex = (1 << level) - 1 + node + groupShift - val binsForNode: Array[Double] = getBinDataForNode(node) - logDebug("nodeImpurityIndex = " + nodeImpurityIndex) - val parentNodeImpurity = parentImpurities(nodeImpurityIndex) - logDebug("parent node impurity = " + parentNodeImpurity) - bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) - node += 1 - } - timer.stop("chooseSplits") - - bestSplits + }.maxBy(_._2.gain) } /** * Get the number of values to be stored per node in the bin aggregates. - * - * @param numBins Number of bins = 1 + number of possible splits. */ - private def getElementsPerNode(metadata: DecisionTreeMetadata, numBins: Int): Int = { + private def getElementsPerNode(metadata: DecisionTreeMetadata): Int = { + val totalBins = metadata.numBins.sum if (metadata.isClassification) { - if (metadata.isMulticlassWithCategoricalFeatures) { - 2 * metadata.numClasses * numBins * metadata.numFeatures - } else { - metadata.numClasses * numBins * metadata.numFeatures - } + metadata.numClasses * totalBins } else { - 3 * numBins * metadata.numFeatures + 3 * totalBins } } @@ -1284,6 +919,7 @@ object DecisionTree extends Serializable with Logging { * Continuous features: * For each feature, there are numBins - 1 possible splits representing the possible binary * decisions at each node in the tree. + * This finds locations (feature values) for splits using a subsample of the data. * * Categorical features: * For each feature, there is 1 bin per split. @@ -1292,7 +928,6 @@ object DecisionTree extends Serializable with Logging { * For multiclass classification with a low-arity feature * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), * the feature is split based on subsets of categories. - * There are (1 << maxFeatureValue - 1) - 1 splits. * (b) "ordered features" * For regression and binary classification, * and for multiclass classification with a high-arity feature, @@ -1302,7 +937,7 @@ object DecisionTree extends Serializable with Logging { * @param metadata Learning and dataset metadata * @return A tuple of (splits, bins). * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] - * of size (numFeatures, numBins - 1). + * of size (numFeatures, numSplits). * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] * of size (numFeatures, numBins). */ @@ -1310,84 +945,80 @@ object DecisionTree extends Serializable with Logging { input: RDD[LabeledPoint], metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { - val count = input.count() + logDebug("isMulticlass = " + metadata.isMulticlass) - // Find the number of features by looking at the first sample - val numFeatures = input.take(1)(0).features.size - - val maxBins = metadata.maxBins - val numBins = if (maxBins <= count) maxBins else count.toInt - logDebug("numBins = " + numBins) - val isMulticlass = metadata.isMulticlass - logDebug("isMulticlass = " + isMulticlass) - - /* - * Ensure numBins is always greater than the categories. For multiclass classification, - * numBins should be greater than 2^(maxCategories - 1) - 1. - * It's a limitation of the current implementation but a reasonable trade-off since features - * with large number of categories get favored over continuous features. - * - * This needs to be checked here instead of in Strategy since numBins can be determined - * by the number of training examples. - * TODO: Allow this case, where we simply will know nothing about some categories. - */ - if (metadata.featureArity.size > 0) { - val maxCategoriesForFeatures = metadata.featureArity.maxBy(_._2)._2 - require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + - "in categorical features") - } - - // Calculate the number of sample for approximate quantile calculation. - val requiredSamples = numBins*numBins - val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 - logDebug("fraction of data used for calculating quantiles = " + fraction) + val numFeatures = metadata.numFeatures - // sampled input for RDD calculation - val sampledInput = + // Sample the input only if there are continuous features. + val hasContinuousFeatures = Range(0, numFeatures).exists(metadata.isContinuous) + val sampledInput = if (hasContinuousFeatures) { + // Calculate the number of samples for approximate quantile calculation. + val requiredSamples = math.max(metadata.maxBins * metadata.maxBins, 10000) + val fraction = if (requiredSamples < metadata.numExamples) { + requiredSamples.toDouble / metadata.numExamples + } else { + 1.0 + } + logDebug("fraction of data used for calculating quantiles = " + fraction) input.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()).collect() - val numSamples = sampledInput.length - - val stride: Double = numSamples.toDouble / numBins - logDebug("stride = " + stride) + } else { + new Array[LabeledPoint](0) + } metadata.quantileStrategy match { case Sort => - val splits = Array.ofDim[Split](numFeatures, numBins - 1) - val bins = Array.ofDim[Bin](numFeatures, numBins) + val splits = new Array[Array[Split]](numFeatures) + val bins = new Array[Array[Bin]](numFeatures) // Find all splits. - // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - // Check whether the feature is continuous. - val isFeatureContinuous = metadata.isContinuous(featureIndex) - if (isFeatureContinuous) { + val numSplits = metadata.numSplits(featureIndex) + val numBins = metadata.numBins(featureIndex) + if (metadata.isContinuous(featureIndex)) { + val numSamples = sampledInput.length + splits(featureIndex) = new Array[Split](numSplits) + bins(featureIndex) = new Array[Bin](numBins) val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted - val stride: Double = numSamples.toDouble / numBins + val stride: Double = numSamples.toDouble / metadata.numBins(featureIndex) logDebug("stride = " + stride) - for (index <- 0 until numBins - 1) { - val sampleIndex = index * stride.toInt + for (splitIndex <- 0 until numSplits) { + val sampleIndex = splitIndex * stride.toInt // Set threshold halfway in between 2 samples. val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 - val split = new Split(featureIndex, threshold, Continuous, List()) - splits(featureIndex)(index) = split + splits(featureIndex)(splitIndex) = + new Split(featureIndex, threshold, Continuous, List()) } - } else { // Categorical feature - val featureCategories = metadata.featureArity(featureIndex) - - // Use different bin/split calculation strategy for categorical features in multiclass - // classification that satisfy the space constraint. + bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), + splits(featureIndex)(0), Continuous, Double.MinValue) + for (splitIndex <- 1 until numSplits) { + bins(featureIndex)(splitIndex) = + new Bin(splits(featureIndex)(splitIndex - 1), splits(featureIndex)(splitIndex), + Continuous, Double.MinValue) + } + bins(featureIndex)(numSplits) = new Bin(splits(featureIndex)(numSplits - 1), + new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) + } else { + // Categorical feature + val featureArity = metadata.featureArity(featureIndex) if (metadata.isUnordered(featureIndex)) { - // 2^(maxFeatureValue- 1) - 1 combinations - var index = 0 - while (index < (1 << featureCategories - 1) - 1) { - val categories: List[Double] - = extractMultiClassCategories(index + 1, featureCategories) - splits(featureIndex)(index) - = new Split(featureIndex, Double.MinValue, Categorical, categories) - bins(featureIndex)(index) = { - if (index == 0) { + // TODO: The second half of the bins are unused. Actually, we could just use + // splits and not build bins for unordered features. That should be part of + // a later PR since it will require changing other code (using splits instead + // of bins in a few places). + // Unordered features + // 2^(maxFeatureValue - 1) - 1 combinations + splits(featureIndex) = new Array[Split](numSplits) + bins(featureIndex) = new Array[Bin](numBins) + var splitIndex = 0 + while (splitIndex < numSplits) { + val categories: List[Double] = + extractMultiClassCategories(splitIndex + 1, featureArity) + splits(featureIndex)(splitIndex) = + new Split(featureIndex, Double.MinValue, Categorical, categories) + bins(featureIndex)(splitIndex) = { + if (splitIndex == 0) { new Bin( new DummyCategoricalSplit(featureIndex, Categorical), splits(featureIndex)(0), @@ -1395,96 +1026,24 @@ object DecisionTree extends Serializable with Logging { Double.MinValue) } else { new Bin( - splits(featureIndex)(index - 1), - splits(featureIndex)(index), + splits(featureIndex)(splitIndex - 1), + splits(featureIndex)(splitIndex), Categorical, Double.MinValue) } } - index += 1 - } - } else { // ordered feature - /* For a given categorical feature, use a subsample of the data - * to choose how to arrange possible splits. - * This examines each category and computes a centroid. - * These centroids are later used to sort the possible splits. - * centroidForCategories is a mapping: category (for the given feature) --> centroid - */ - val centroidForCategories = { - if (isMulticlass) { - // For categorical variables in multiclass classification, - // each bin is a category. The bins are sorted and they - // are ordered by calculating the impurity of their corresponding labels. - sampledInput.map(lp => (lp.features(featureIndex), lp.label)) - .groupBy(_._1) - .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) - .map(x => (x._1, x._2.values.toArray)) - .map(x => (x._1, metadata.impurity.calculate(x._2, x._2.sum))) - } else { // regression or binary classification - // For categorical variables in regression and binary classification, - // each bin is a category. The bins are sorted and they - // are ordered by calculating the centroid of their corresponding labels. - sampledInput.map(lp => (lp.features(featureIndex), lp.label)) - .groupBy(_._1) - .mapValues(x => x.map(_._2).sum / x.map(_._1).length) - } - } - - logDebug("centroid for categories = " + centroidForCategories.mkString(",")) - - // Check for missing categorical variables and putting them last in the sorted list. - val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() - for (i <- 0 until featureCategories) { - if (centroidForCategories.contains(i)) { - fullCentroidForCategories(i) = centroidForCategories(i) - } else { - fullCentroidForCategories(i) = Double.MaxValue - } - } - - // bins sorted by centroids - val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) - - logDebug("centroid for categorical variable = " + categoriesSortedByCentroid) - - var categoriesForSplit = List[Double]() - categoriesSortedByCentroid.iterator.zipWithIndex.foreach { - case ((key, value), index) => - categoriesForSplit = key :: categoriesForSplit - splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, - Categorical, categoriesForSplit) - bins(featureIndex)(index) = { - if (index == 0) { - new Bin(new DummyCategoricalSplit(featureIndex, Categorical), - splits(featureIndex)(0), Categorical, key) - } else { - new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Categorical, key) - } - } + splitIndex += 1 } + } else { + // Ordered features + // Bins correspond to feature values, so we do not need to compute splits or bins + // beforehand. Splits are constructed as needed during training. + splits(featureIndex) = new Array[Split](0) + bins(featureIndex) = new Array[Bin](0) } } featureIndex += 1 } - - // Find all bins. - featureIndex = 0 - while (featureIndex < numFeatures) { - val isFeatureContinuous = metadata.isContinuous(featureIndex) - if (isFeatureContinuous) { // Bins for categorical variables are already assigned. - bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), - splits(featureIndex)(0), Continuous, Double.MinValue) - for (index <- 1 until numBins - 1) { - val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Continuous, Double.MinValue) - bins(featureIndex)(index) = bin - } - bins(featureIndex)(numBins-1) = new Bin(splits(featureIndex)(numBins-2), - new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) - } - featureIndex += 1 - } (splits, bins) case MinMax => throw new UnsupportedOperationException("minmax not supported yet.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala new file mode 100644 index 0000000000000..866d85a79bea1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -0,0 +1,213 @@ +/* + * 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.tree.impurity._ + +/** + * DecisionTree statistics aggregator. + * This holds a flat array of statistics for a set of (nodes, features, bins) + * and helps with indexing. + */ +private[tree] class DTStatsAggregator( + val metadata: DecisionTreeMetadata, + val numNodes: Int) extends Serializable { + + /** + * [[ImpurityAggregator]] instance specifying the impurity type. + */ + val impurityAggregator: ImpurityAggregator = metadata.impurity match { + case Gini => new GiniAggregator(metadata.numClasses) + case Entropy => new EntropyAggregator(metadata.numClasses) + case Variance => new VarianceAggregator() + case _ => throw new IllegalArgumentException(s"Bad impurity parameter: ${metadata.impurity}") + } + + /** + * Number of elements (Double values) used for the sufficient statistics of each bin. + */ + val statsSize: Int = impurityAggregator.statsSize + + val numFeatures: Int = metadata.numFeatures + + /** + * Number of bins for each feature. This is indexed by the feature index. + */ + val numBins: Array[Int] = metadata.numBins + + /** + * Number of splits for the given feature. + */ + def numSplits(featureIndex: Int): Int = metadata.numSplits(featureIndex) + + /** + * Indicator for each feature of whether that feature is an unordered feature. + * TODO: Is Array[Boolean] any faster? + */ + def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) + + /** + * Offset for each feature for calculating indices into the [[allStats]] array. + */ + private val featureOffsets: Array[Int] = { + def featureOffsetsCalc(total: Int, featureIndex: Int): Int = { + if (isUnordered(featureIndex)) { + total + 2 * numBins(featureIndex) + } else { + total + numBins(featureIndex) + } + } + Range(0, numFeatures).scanLeft(0)(featureOffsetsCalc).map(statsSize * _).toArray + } + + /** + * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. + */ + private val nodeStride: Int = featureOffsets.last + + /** + * Total number of elements stored in this aggregator. + */ + val allStatsSize: Int = numNodes * nodeStride + + /** + * Flat array of elements. + * Index for start of stats for a (node, feature, bin) is: + * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, the left child stats have binIndex in [0, numBins(featureIndex)) + * and the right child stats in [numBins(featureIndex), 2 * numBins(featureIndex)) + */ + val allStats: Array[Double] = new Array[Double](allStatsSize) + + /** + * Get an [[ImpurityCalculator]] for a given (node, feature, bin). + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + */ + def getImpurityCalculator(nodeFeatureOffset: Int, binIndex: Int): ImpurityCalculator = { + impurityAggregator.getCalculator(allStats, nodeFeatureOffset + binIndex * statsSize) + } + + /** + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + */ + def update(nodeIndex: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { + val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label) + } + + /** + * Pre-compute node offset for use with [[nodeUpdate]]. + */ + def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. + */ + def nodeUpdate(nodeOffset: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { + val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For ordered features only. + */ + def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { + require(!isUnordered(featureIndex), + s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" + + s" for unordered feature $featureIndex.") + nodeIndex * nodeStride + featureOffsets(featureIndex) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For unordered features only. + */ + def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { + require(isUnordered(featureIndex), + s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + + s" but was called for ordered feature $featureIndex.") + val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) + (baseOffset, baseOffset + numBins(featureIndex) * statsSize) + } + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin), using the given label. + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + */ + def nodeFeatureUpdate(nodeFeatureOffset: Int, binIndex: Int, label: Double): Unit = { + impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label) + } + + /** + * For a given (node, feature), merge the stats for two bins. + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + * @param binIndex The other bin is merged into this bin. + * @param otherBinIndex This bin is not modified. + */ + def mergeForNodeFeature(nodeFeatureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { + impurityAggregator.merge(allStats, nodeFeatureOffset + binIndex * statsSize, + nodeFeatureOffset + otherBinIndex * statsSize) + } + + /** + * Merge this aggregator with another, and returns this aggregator. + * This method modifies this aggregator in-place. + */ + def merge(other: DTStatsAggregator): DTStatsAggregator = { + require(allStatsSize == other.allStatsSize, + s"DTStatsAggregator.merge requires that both aggregators have the same length stats vectors." + + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") + var i = 0 + // TODO: Test BLAS.axpy + while (i < allStatsSize) { + allStats(i) += other.allStats(i) + i += 1 + } + this + } + +} + +private[tree] object DTStatsAggregator extends Serializable { + + /** + * Combines two aggregates (modifying the first) and returns the combination. + */ + def binCombOp( + agg1: DTStatsAggregator, + agg2: DTStatsAggregator): DTStatsAggregator = { + agg1.merge(agg2) + } + +} 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 d9eda354dc986..e95add7558bcf 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 @@ -26,14 +26,15 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.rdd.RDD - /** * Learning and dataset metadata for DecisionTree. * * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. * For regression: fixed at 0 (no meaning). + * @param maxBins Maximum number of bins, for all features. * @param featureArity Map: categorical feature index --> arity. * I.e., the feature takes values in {0, ..., arity - 1}. + * @param numBins Number of bins for each feature. */ private[tree] class DecisionTreeMetadata( val numFeatures: Int, @@ -42,6 +43,7 @@ private[tree] class DecisionTreeMetadata( val maxBins: Int, val featureArity: Map[Int, Int], val unorderedFeatures: Set[Int], + val numBins: Array[Int], val impurity: Impurity, val quantileStrategy: QuantileStrategy) extends Serializable { @@ -57,10 +59,26 @@ private[tree] class DecisionTreeMetadata( def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + /** + * Number of splits for the given feature. + * For unordered features, there are 2 bins per split. + * For ordered features, there is 1 more bin than split. + */ + def numSplits(featureIndex: Int): Int = if (isUnordered(featureIndex)) { + numBins(featureIndex) >> 1 + } else { + numBins(featureIndex) - 1 + } + } private[tree] object DecisionTreeMetadata { + /** + * Construct a [[DecisionTreeMetadata]] instance for this dataset and parameters. + * This computes which categorical features will be ordered vs. unordered, + * as well as the number of splits and bins for each feature. + */ def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { val numFeatures = input.take(1)(0).features.size @@ -70,32 +88,55 @@ private[tree] object DecisionTreeMetadata { case Regression => 0 } - val maxBins = math.min(strategy.maxBins, numExamples).toInt - val log2MaxBinsp1 = math.log(maxBins + 1) / math.log(2.0) + val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt + + // 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 + require(maxCategoriesPerFeature <= maxPossibleBins, + s"DecisionTree requires maxBins (= $maxPossibleBins) >= max categories " + + s"in categorical features (= $maxCategoriesPerFeature)") + } val unorderedFeatures = new mutable.HashSet[Int]() + val numBins = Array.fill[Int](numFeatures)(maxPossibleBins) if (numClasses > 2) { - strategy.categoricalFeaturesInfo.foreach { case (f, k) => - if (k - 1 < log2MaxBinsp1) { - // Note: The above check is equivalent to checking: - // numUnorderedBins = (1 << k - 1) - 1 < maxBins - unorderedFeatures.add(f) + // Multiclass classification + val maxCategoriesForUnorderedFeature = + ((math.log(maxPossibleBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + // 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 { - // TODO: Allow this case, where we simply will know nothing about some categories? - require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + - s"in categorical features (>= $k)") + numBins(featureIndex) = numCategories } } } else { - strategy.categoricalFeaturesInfo.foreach { case (f, k) => - require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + - s"in categorical features (>= $k)") + // Binary classification or regression + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + numBins(featureIndex) = numCategories } } - new DecisionTreeMetadata(numFeatures, numExamples, numClasses, maxBins, - strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, + strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, strategy.impurity, strategy.quantileCalculationStrategy) } + /** + * 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) + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala index 170e43e222083..35e361ae309cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -48,54 +48,63 @@ private[tree] object TreePoint { * binning feature values in preparation for DecisionTree training. * @param input Input dataset. * @param bins Bins for features, of size (numFeatures, numBins). - * @param metadata Learning and dataset metadata + * @param metadata Learning and dataset metadata * @return TreePoint dataset representation */ def convertToTreeRDD( input: RDD[LabeledPoint], bins: Array[Array[Bin]], metadata: DecisionTreeMetadata): RDD[TreePoint] = { + // Construct arrays for featureArity and isUnordered for efficiency in the inner loop. + val featureArity: Array[Int] = new Array[Int](metadata.numFeatures) + val isUnordered: Array[Boolean] = new Array[Boolean](metadata.numFeatures) + var featureIndex = 0 + while (featureIndex < metadata.numFeatures) { + featureArity(featureIndex) = metadata.featureArity.getOrElse(featureIndex, 0) + isUnordered(featureIndex) = metadata.isUnordered(featureIndex) + featureIndex += 1 + } input.map { x => - TreePoint.labeledPointToTreePoint(x, bins, metadata) + TreePoint.labeledPointToTreePoint(x, bins, featureArity, isUnordered) } } /** * Convert one LabeledPoint into its TreePoint representation. * @param bins Bins for features, of size (numFeatures, numBins). + * @param featureArity Array indexed by feature, with value 0 for continuous and numCategories + * for categorical features. + * @param isUnordered Array index by feature, with value true for unordered categorical features. */ private def labeledPointToTreePoint( labeledPoint: LabeledPoint, bins: Array[Array[Bin]], - metadata: DecisionTreeMetadata): TreePoint = { - + featureArity: Array[Int], + isUnordered: Array[Boolean]): TreePoint = { val numFeatures = labeledPoint.features.size - val numBins = bins(0).size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { - arr(featureIndex) = findBin(featureIndex, labeledPoint, metadata.isContinuous(featureIndex), - metadata.isUnordered(featureIndex), bins, metadata.featureArity) + arr(featureIndex) = findBin(featureIndex, labeledPoint, featureArity(featureIndex), + isUnordered(featureIndex), bins) featureIndex += 1 } - new TreePoint(labeledPoint.label, arr) } /** * Find bin for one (labeledPoint, feature). * + * @param featureArity 0 for continuous features; number of categories for categorical features. * @param isUnorderedFeature (only applies if feature is categorical) * @param bins Bins for features, of size (numFeatures, numBins). - * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity */ private def findBin( featureIndex: Int, labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean, + featureArity: Int, isUnorderedFeature: Boolean, - bins: Array[Array[Bin]], - categoricalFeaturesInfo: Map[Int, Int]): Int = { + bins: Array[Array[Bin]]): Int = { /** * Binary search helper method for continuous feature. @@ -121,44 +130,7 @@ private[tree] object TreePoint { -1 } - /** - * Sequential search helper method to find bin for categorical feature in multiclass - * classification. The category is returned since each category can belong to multiple - * splits. The actual left/right child allocation per split is performed in the - * sequential phase of the bin aggregate operation. - */ - def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { - labeledPoint.features(featureIndex).toInt - } - - /** - * Sequential search helper method to find bin for categorical feature - * (for classification and regression). - */ - def sequentialBinSearchForOrderedCategoricalFeature(): Int = { - val featureCategories = categoricalFeaturesInfo(featureIndex) - val featureValue = labeledPoint.features(featureIndex) - var binIndex = 0 - while (binIndex < featureCategories) { - val bin = bins(featureIndex)(binIndex) - val categories = bin.highSplit.categories - if (categories.contains(featureValue)) { - return binIndex - } - binIndex += 1 - } - if (featureValue < 0 || featureValue >= featureCategories) { - throw new IllegalArgumentException( - s"DecisionTree given invalid data:" + - s" Feature $featureIndex is categorical with values in" + - s" {0,...,${featureCategories - 1}," + - s" but a data point gives it value $featureValue.\n" + - " Bad data point: " + labeledPoint.toString) - } - -1 - } - - if (isFeatureContinuous) { + if (featureArity == 0) { // Perform binary search for finding bin for continuous features. val binIndex = binarySearchForBins() if (binIndex == -1) { @@ -168,18 +140,17 @@ private[tree] object TreePoint { } binIndex } else { - // Perform sequential search to find bin for categorical features. - val binIndex = if (isUnorderedFeature) { - sequentialBinSearchForUnorderedCategoricalFeatureInClassification() - } else { - sequentialBinSearchForOrderedCategoricalFeature() - } - if (binIndex == -1) { - throw new RuntimeException("No bin was found for categorical feature." + - " This error can occur when given invalid data values (such as NaN)." + - s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + // Categorical feature bins are indexed by feature values. + val featureValue = labeledPoint.features(featureIndex) + if (featureValue < 0 || featureValue >= featureArity) { + throw new IllegalArgumentException( + s"DecisionTree given invalid data:" + + s" Feature $featureIndex is categorical with values in" + + s" {0,...,${featureArity - 1}," + + s" but a data point gives it value $featureValue.\n" + + " Bad data point: " + labeledPoint.toString) } - binIndex + featureValue.toInt } } } 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 96d2471e1f88c..1c8afc2d0f4bc 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 @@ -74,3 +74,87 @@ object Entropy extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param numClasses Number of classes for label. + */ +private[tree] class EntropyAggregator(numClasses: Int) + extends ImpurityAggregator(numClasses) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"EntropyAggregator given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + allStats(offset + label.toInt) += 1 + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): EntropyCalculator = { + new EntropyCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[EntropyAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class EntropyCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: EntropyCalculator = new EntropyCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Entropy.calculate(stats, stats.sum) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats.sum.toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + indexOfLargestArrayElement(stats) + } + + /** + * Probability of the label given by [[predict]]. + */ + override def prob(label: Double): Double = { + val lbl = label.toInt + require(lbl < stats.length, + s"EntropyCalculator.prob given invalid label: $lbl (should be < ${stats.length}") + val cnt = count + if (cnt == 0) { + 0 + } else { + stats(lbl) / cnt + } + } + + override def toString: String = s"EntropyCalculator(stats = [${stats.mkString(", ")}])" + +} 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 d586f449048bb..5cfdf345d163c 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 @@ -70,3 +70,87 @@ object Gini extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param numClasses Number of classes for label. + */ +private[tree] class GiniAggregator(numClasses: Int) + extends ImpurityAggregator(numClasses) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"GiniAggregator given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + allStats(offset + label.toInt) += 1 + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): GiniCalculator = { + new GiniCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[GiniAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class GiniCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: GiniCalculator = new GiniCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Gini.calculate(stats, stats.sum) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats.sum.toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + indexOfLargestArrayElement(stats) + } + + /** + * Probability of the label given by [[predict]]. + */ + override def prob(label: Double): Double = { + val lbl = label.toInt + require(lbl < stats.length, + s"GiniCalculator.prob given invalid label: $lbl (should be < ${stats.length}") + val cnt = count + if (cnt == 0) { + 0 + } else { + stats(lbl) / cnt + } + } + + override def toString: String = s"GiniCalculator(stats = [${stats.mkString(", ")}])" + +} 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 92b0c7b4a6fbc..5a047d6cb5480 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 @@ -22,6 +22,9 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: * Trait for calculating information gain. + * This trait is used for + * (a) setting the impurity parameter in [[org.apache.spark.mllib.tree.configuration.Strategy]] + * (b) calculating impurity values from sufficient statistics. */ @Experimental trait Impurity extends Serializable { @@ -47,3 +50,127 @@ trait Impurity extends Serializable { @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double } + +/** + * Interface for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param statsSize Length of the vector of sufficient statistics for one bin. + */ +private[tree] abstract class ImpurityAggregator(val statsSize: Int) extends Serializable { + + /** + * Merge the stats from one bin into another. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for (node, feature, bin) which is modified by the merge. + * @param otherOffset Start index of stats for (node, feature, other bin) which is not modified. + */ + def merge(allStats: Array[Double], offset: Int, otherOffset: Int): Unit = { + var i = 0 + while (i < statsSize) { + allStats(offset + i) += allStats(otherOffset + i) + i += 1 + } + } + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): ImpurityCalculator + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[ImpurityAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] abstract class ImpurityCalculator(val stats: Array[Double]) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: ImpurityCalculator + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double + + /** + * Add the stats from another calculator into this one, modifying and returning this calculator. + */ + def add(other: ImpurityCalculator): ImpurityCalculator = { + require(stats.size == other.stats.size, + s"Two ImpurityCalculator instances cannot be added with different counts sizes." + + s" Sizes are ${stats.size} and ${other.stats.size}.") + var i = 0 + while (i < other.stats.size) { + stats(i) += other.stats(i) + i += 1 + } + this + } + + /** + * Subtract the stats from another calculator from this one, modifying and returning this + * calculator. + */ + def subtract(other: ImpurityCalculator): ImpurityCalculator = { + require(stats.size == other.stats.size, + s"Two ImpurityCalculator instances cannot be subtracted with different counts sizes." + + s" Sizes are ${stats.size} and ${other.stats.size}.") + var i = 0 + while (i < other.stats.size) { + stats(i) -= other.stats(i) + i += 1 + } + this + } + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double + + /** + * Probability of the label given by [[predict]], or -1 if no probability is available. + */ + def prob(label: Double): Double = -1 + + /** + * Return the index of the largest array element. + * Fails if the array is empty. + */ + protected def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } + } + if (result._1 < 0) { + throw new RuntimeException("ImpurityCalculator internal error:" + + " indexOfLargestArrayElement failed") + } + result._1 + } + +} 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 f7d99a40eb380..e9ccecb1b8067 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 @@ -61,3 +61,75 @@ object Variance extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + */ +private[tree] class VarianceAggregator() + extends ImpurityAggregator(statsSize = 3) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + allStats(offset) += 1 + allStats(offset + 1) += label + allStats(offset + 2) += label * label + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): VarianceCalculator = { + new VarianceCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[GiniAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + require(stats.size == 3, + s"VarianceCalculator requires sufficient statistics array stats to be of length 3," + + s" but was given array of length ${stats.size}.") + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: VarianceCalculator = new VarianceCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Variance.calculate(stats(0), stats(1), stats(2)) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats(0).toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + stats(1) / count + } + + override def toString: String = { + s"VarianceAggregator(cnt = ${stats(0)}, sum = ${stats(1)}, sum2 = ${stats(2)})" + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index af35d88f713e5..0cad473782af1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * Used for "binning" the features bins for faster best split calculation. + * Used for "binning" the feature values for faster best split calculation. * * For a continuous feature, the bin is determined by a low and a high split, * where an example with featureValue falls into the bin s.t. @@ -30,13 +30,16 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * bins, splits, and feature values. The bin is determined by category/feature value. * However, the bins are not necessarily ordered by feature value; * they are ordered using impurity. + * * For unordered categorical features, there is a 1-1 correspondence between bins, splits, * where bins and splits correspond to subsets of feature values (in highSplit.categories). + * An unordered feature with k categories uses (1 << k - 1) - 1 bins, corresponding to all + * partitionings of categories into 2 disjoint, non-empty sets. * * @param lowSplit signifying the lower threshold for the continuous feature to be * accepted in the bin * @param highSplit signifying the upper threshold for the continuous feature to be - * accepted in the bin + * accepted in the bin * @param featureType type of feature -- categorical or continuous * @param category categorical label value accepted in the bin for ordered features */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 0eee6262781c1..5b8a4cbed2306 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -24,8 +24,13 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * Node in a decision tree - * @param id integer node id + * Node in a decision tree. + * + * About node indexing: + * Nodes are indexed from 1. Node 1 is the root; nodes 2, 3 are the left, right children. + * Node index 0 is not used. + * + * @param id integer node id, from 1 * @param predict predicted value at the node * @param isLeaf whether the leaf is a node * @param split split to calculate left and right nodes @@ -51,17 +56,13 @@ class Node ( * @param nodes array of nodes */ def build(nodes: Array[Node]): Unit = { - - logDebug("building node " + id + " at level " + - (scala.math.log(id + 1)/scala.math.log(2)).toInt ) + logDebug("building node " + id + " at level " + Node.indexToLevel(id)) logDebug("id = " + id + ", split = " + split) logDebug("stats = " + stats) logDebug("predict = " + predict) if (!isLeaf) { - val leftNodeIndex = id * 2 + 1 - val rightNodeIndex = id * 2 + 2 - leftNode = Some(nodes(leftNodeIndex)) - rightNode = Some(nodes(rightNodeIndex)) + leftNode = Some(nodes(Node.leftChildIndex(id))) + rightNode = Some(nodes(Node.rightChildIndex(id))) leftNode.get.build(nodes) rightNode.get.build(nodes) } @@ -96,24 +97,20 @@ class Node ( * Get the number of nodes in tree below this node, including leaf nodes. * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. */ - private[tree] def numDescendants: Int = { - if (isLeaf) { - 0 - } else { - 2 + leftNode.get.numDescendants + rightNode.get.numDescendants - } + private[tree] def numDescendants: Int = if (isLeaf) { + 0 + } else { + 2 + leftNode.get.numDescendants + rightNode.get.numDescendants } /** * Get depth of tree from this node. * E.g.: Depth 0 means this is a leaf node. */ - private[tree] def subtreeDepth: Int = { - if (isLeaf) { - 0 - } else { - 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) - } + private[tree] def subtreeDepth: Int = if (isLeaf) { + 0 + } else { + 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) } /** @@ -148,3 +145,49 @@ class Node ( } } + +private[tree] object Node { + + /** + * Return the index of the left child of this node. + */ + def leftChildIndex(nodeIndex: Int): Int = nodeIndex << 1 + + /** + * Return the index of the right child of this node. + */ + def rightChildIndex(nodeIndex: Int): Int = (nodeIndex << 1) + 1 + + /** + * Get the parent index of the given node, or 0 if it is the root. + */ + def parentIndex(nodeIndex: Int): Int = nodeIndex >> 1 + + /** + * Return the level of a tree which the given node is in. + */ + def indexToLevel(nodeIndex: Int): Int = if (nodeIndex == 0) { + throw new IllegalArgumentException(s"0 is not a valid node index.") + } else { + java.lang.Integer.numberOfTrailingZeros(java.lang.Integer.highestOneBit(nodeIndex)) + } + + /** + * Returns true if this is a left child. + * Note: Returns false for the root. + */ + def isLeftChild(nodeIndex: Int): Boolean = nodeIndex > 1 && nodeIndex % 2 == 0 + + /** + * Return the maximum number of nodes which can be in the given level of the tree. + * @param level Level of tree (0 = root). + */ + def maxNodesInLevel(level: Int): Int = 1 << level + + /** + * Return the index of the first node in the given level. + * @param level Level of tree (0 = root). + */ + def startIndexInLevel(level: Int): Int = 1 << level + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 2f36fd907772c..8e556c917b2e7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -21,15 +21,16 @@ import scala.collection.JavaConverters._ import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.regression.LabeledPoint + class DecisionTreeSuite extends FunSuite with LocalSparkContext { @@ -59,12 +60,13 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") } - test("split and bin calculation") { + test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) @@ -72,7 +74,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) } - test("split and bin calculation for categorical variables") { + test("Binary classification with binary (ordered) categorical features:" + + " split and bin calculation") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -83,77 +86,20 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) assert(splits.length === 2) assert(bins.length === 2) - assert(splits(0).length === 99) - assert(bins(0).length === 100) - - // Check splits. - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(0)(1).categories.contains(0.0)) - - assert(splits(0)(2) === null) - - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 2) - assert(splits(1)(1).categories.contains(1.0)) - assert(splits(1)(1).categories.contains(0.0)) - - assert(splits(1)(2) === null) - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - - assert(bins(0)(1).category === 0.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(0.0)) - - assert(bins(0)(2) === null) - - assert(bins(1)(0).category === 0.0) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(1)(1).category === 1.0) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 2) - assert(bins(1)(1).highSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(1)(2) === null) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } - test("split and bin calculations for categorical variables with no sample for one category") { + test("Binary classification with 3-ary (ordered) categorical features," + + " with no samples for one category") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -164,104 +110,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - - // Check splits. - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(0)(1).categories.contains(0.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 3) - assert(splits(0)(2).categories.contains(1.0)) - assert(splits(0)(2).categories.contains(0.0)) - assert(splits(0)(2).categories.contains(2.0)) - - assert(splits(0)(3) === null) - - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 2) - assert(splits(1)(1).categories.contains(1.0)) - assert(splits(1)(1).categories.contains(0.0)) - - assert(splits(1)(2).feature === 1) - assert(splits(1)(2).threshold === Double.MinValue) - assert(splits(1)(2).featureType === Categorical) - assert(splits(1)(2).categories.length === 3) - assert(splits(1)(2).categories.contains(1.0)) - assert(splits(1)(2).categories.contains(0.0)) - assert(splits(1)(2).categories.contains(2.0)) - - assert(splits(1)(3) === null) - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - - assert(bins(0)(1).category === 0.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(0.0)) - - assert(bins(0)(2).category === 2.0) - assert(bins(0)(2).lowSplit.categories.length === 2) - assert(bins(0)(2).lowSplit.categories.contains(1.0)) - assert(bins(0)(2).lowSplit.categories.contains(0.0)) - assert(bins(0)(2).highSplit.categories.length === 3) - assert(bins(0)(2).highSplit.categories.contains(1.0)) - assert(bins(0)(2).highSplit.categories.contains(0.0)) - assert(bins(0)(2).highSplit.categories.contains(2.0)) - - assert(bins(0)(3) === null) - - assert(bins(1)(0).category === 0.0) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(1)(1).category === 1.0) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 2) - assert(bins(1)(1).highSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(1)(2).category === 2.0) - assert(bins(1)(2).lowSplit.categories.length === 2) - assert(bins(1)(2).lowSplit.categories.contains(0.0)) - assert(bins(1)(2).lowSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.length === 3) - assert(bins(1)(2).highSplit.categories.contains(0.0)) - assert(bins(1)(2).highSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.contains(2.0)) - - assert(bins(1)(3) === null) + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } test("extract categories from a number for multiclass classification") { @@ -270,8 +128,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) } - test("split and bin calculations for unordered categorical variables with multiclass " + - "classification") { + test("Multiclass classification with unordered categorical features:" + + " split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -282,8 +140,15 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(bins.length === 2) + assert(splits(0).length === 3) + assert(bins(0).length === 6) // Expecting 2^2 - 1 = 3 bins/splits assert(splits(0)(0).feature === 0) @@ -321,10 +186,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(1)(2).categories.contains(0.0)) assert(splits(1)(2).categories.contains(1.0)) - assert(splits(0)(3) === null) - assert(splits(1)(3) === null) - - // Check bins. assert(bins(0)(0).category === Double.MinValue) @@ -360,13 +221,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(1)(2).highSplit.categories.contains(1.0)) assert(bins(1)(2).highSplit.categories.contains(0.0)) - assert(bins(0)(3) === null) - assert(bins(1)(3) === null) - } - test("split and bin calculations for ordered categorical variables with multiclass " + - "classification") { + test("Multiclass classification with ordered categorical features: split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() assert(arr.length === 3000) val rdd = sc.parallelize(arr) @@ -377,52 +234,21 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) + // 2^10 - 1 > 100, so categorical features will be ordered + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - - // 2^10 - 1 > 100, so categorical variables will be ordered - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(2.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 3) - assert(splits(0)(2).categories.contains(2.0)) - assert(splits(0)(2).categories.contains(1.0)) - - assert(splits(0)(10) === null) - assert(splits(1)(10) === null) - - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - assert(bins(0)(1).category === 2.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(2.0)) - - assert(bins(0)(10) === null) - + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } - test("classification stump with all categorical variables") { + test("Binary classification stump with ordered categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -433,15 +259,23 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 - assert(split.categories.length === 1) - assert(split.categories.contains(1.0)) + assert(split.categories === List(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) @@ -452,7 +286,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } - test("regression stump with all categorical variables") { + test("Regression stump with 3-ary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -462,10 +296,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 @@ -480,7 +318,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } - test("regression stump with categorical variables of arity 2") { + test("Regression stump with binary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -490,6 +328,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) validateRegressor(model, arr, 0.0) @@ -497,12 +338,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) } - test("stump with fixed label 0 for Gini") { + test("Binary classification stump with fixed label 0 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val strategy = new Strategy(Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -512,7 +357,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -521,12 +366,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.rightImpurity === 0) } - test("stump with fixed label 1 for Gini") { + test("Binary classification stump with fixed label 1 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val strategy = new Strategy(Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -536,7 +385,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -546,12 +395,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("stump with fixed label 0 for Entropy") { + test("Binary classification stump with fixed label 0 for Entropy") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val strategy = new Strategy(Classification, Entropy, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -561,7 +414,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -571,12 +424,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 0) } - test("stump with fixed label 1 for Entropy") { + test("Binary classification stump with fixed label 1 for Entropy") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val strategy = new Strategy(Classification, Entropy, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -586,7 +443,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -596,7 +453,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("second level node building with/without groups") { + test("Second level node building with vs. without groups") { val arr = DecisionTreeSuite.generateOrderedLabeledPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -613,12 +470,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val nodes: Array[Node] = new Array[Node](7) - nodes(0) = modelOneNode.topNode - nodes(0).leftNode = None - nodes(0).rightNode = None + val nodes: Array[Node] = new Array[Node](8) + nodes(1) = modelOneNode.topNode + nodes(1).leftNode = None + nodes(1).rightNode = None - val parentImpurities = Array(0.5, 0.5, 0.5) + val parentImpurities = Array(0, 0.5, 0.5, 0.5) // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) @@ -648,16 +505,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } } - test("stump with categorical variables for multiclass classification") { + test("Multiclass classification stump with 3-ary (unordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -668,7 +528,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } - test("stump with 1 continuous variable for binary classification, to check off-by-1 error") { + test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) @@ -684,26 +544,27 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) } - test("stump with 2 continuous variables for binary classification") { + test("Binary classification stump with 2 continuous features") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) assert(model.topNode.split.get.feature === 1) } - test("stump with categorical variables for multiclass classification, with just enough bins") { - val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features + test("Multiclass classification stump with unordered categorical features," + + " with just enough bins") { + val maxBins = 2 * (math.pow(2, 3 - 1).toInt - 1) // just enough bins to allow unordered features val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -711,6 +572,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) @@ -719,7 +582,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -733,7 +596,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(gain.rightImpurity === 0) } - test("stump with continuous variables for multiclass classification") { + test("Multiclass classification stump with continuous features") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -746,7 +609,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -759,20 +622,21 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } - test("stump with continuous + categorical variables for multiclass classification") { + test("Multiclass classification stump with continuous + unordered categorical features") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -784,17 +648,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.threshold < 2020) } - test("stump with categorical variables for ordered multiclass classification") { + test("Multiclass classification stump with 10-ary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -805,6 +671,18 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } + test("Multiclass classification tree with 10-ary (ordered) categorical features," + + " with just enough bins") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, maxBins = 10, + categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + assert(strategy.isMulticlassClassification) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 0.6) + } } @@ -899,5 +777,4 @@ object DecisionTreeSuite { arr } - } From e16a8e7db5a3b1065b14baf89cb723a59b99226b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 8 Sep 2014 10:24:15 -0700 Subject: [PATCH 02/76] SPARK-3337 Paranoid quoting in shell to allow install dirs with spaces within. ... Tested ! TBH, it isn't a great idea to have directory with spaces within. Because emacs doesn't like it then hadoop doesn't like it. and so on... Author: Prashant Sharma Closes #2229 from ScrapCodes/SPARK-3337/quoting-shell-scripts and squashes the following commits: d4ad660 [Prashant Sharma] SPARK-3337 Paranoid quoting in shell to allow install dirs with spaces within. --- bin/beeline | 2 +- bin/compute-classpath.sh | 12 ++++++------ bin/load-spark-env.sh | 4 ++-- bin/pyspark | 20 ++++++++++---------- bin/run-example | 8 ++++---- bin/spark-class | 20 ++++++++++---------- bin/spark-shell | 10 +++++----- bin/spark-sql | 8 ++++---- bin/spark-submit | 4 ++-- dev/check-license | 16 ++++++++-------- dev/lint-python | 6 +++--- dev/mima | 4 ++-- dev/run-tests | 2 +- dev/scalastyle | 2 +- make-distribution.sh | 2 +- python/run-tests | 6 ++++-- sbin/slaves.sh | 12 ++++++------ sbin/spark-config.sh | 16 ++++++++-------- sbin/spark-daemon.sh | 20 ++++++++++---------- sbin/spark-executor | 8 ++++---- sbin/start-all.sh | 4 ++-- sbin/start-history-server.sh | 4 ++-- sbin/start-master.sh | 4 ++-- sbin/start-slave.sh | 4 ++-- sbin/start-slaves.sh | 12 ++++++------ sbin/start-thriftserver.sh | 8 ++++---- sbin/stop-all.sh | 4 ++-- sbin/stop-history-server.sh | 4 ++-- sbt/sbt | 20 ++++++++++---------- sbt/sbt-launch-lib.bash | 12 ++++++------ 30 files changed, 130 insertions(+), 128 deletions(-) diff --git a/bin/beeline b/bin/beeline index 1bda4dba50605..3fcb6df34339d 100755 --- a/bin/beeline +++ b/bin/beeline @@ -24,7 +24,7 @@ set -o posix # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.hive.beeline.BeeLine" exec "$FWDIR/bin/spark-class" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 16b794a1592e8..15c6779402994 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -23,9 +23,9 @@ SCALA_VERSION=2.10 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" @@ -63,7 +63,7 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l) +num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l)" if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $assembly_folder" echo "You need to build Spark before running this program." @@ -77,7 +77,7 @@ if [ "$num_jars" -gt "1" ]; then exit 1 fi -ASSEMBLY_JAR=$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null) +ASSEMBLY_JAR="$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null)" # Verify that versions of java used to build the jars and run Spark are compatible jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) @@ -103,8 +103,8 @@ else datanucleus_dir="$FWDIR"/lib_managed/jars fi -datanucleus_jars=$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar") -datanucleus_jars=$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g) +datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar")" +datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" if [ -n "$datanucleus_jars" ]; then hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 493d3785a081b..6d4231b204595 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -25,9 +25,9 @@ if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 # Returns the parent of the directory this script lives in. - parent_dir="$(cd `dirname $0`/..; pwd)" + parent_dir="$(cd "`dirname "$0"`"/..; pwd)" - user_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + user_conf_dir="${SPARK_CONF_DIR:-"$parent_dir"/conf}" if [ -f "${user_conf_dir}/spark-env.sh" ]; then # Promote all variable declarations to environment (exported) variables diff --git a/bin/pyspark b/bin/pyspark index 26a16dd600b7a..5142411e36974 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,18 +18,18 @@ # # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -source $FWDIR/bin/utils.sh +source "$FWDIR/bin/utils.sh" SCALA_VERSION=2.10 function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } @@ -48,7 +48,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh # Figure out which Python executable to use if [[ -z "$PYSPARK_PYTHON" ]]; then @@ -57,12 +57,12 @@ fi export PYSPARK_PYTHON # Add the PySpark classes to the Python path: -export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH" +export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: -export OLD_PYTHONSTARTUP=$PYTHONSTARTUP -export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py +export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" +export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" # If IPython options are specified, assume user wants to run IPython if [[ -n "$IPYTHON_OPTS" ]]; then @@ -99,10 +99,10 @@ fi if [[ "$1" =~ \.py$ ]]; then echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 echo -e "Use ./bin/spark-submit \n" 1>&2 - primary=$1 + primary="$1" shift gatherSparkSubmitOpts "$@" - exec $FWDIR/bin/spark-submit "${SUBMISSION_OPTS[@]}" $primary "${APPLICATION_OPTS[@]}" + exec "$FWDIR"/bin/spark-submit "${SUBMISSION_OPTS[@]}" "$primary" "${APPLICATION_OPTS[@]}" else # PySpark shell requires special handling downstream export PYSPARK_SHELL=1 diff --git a/bin/run-example b/bin/run-example index 68a35702eddd3..34dd71c71880e 100755 --- a/bin/run-example +++ b/bin/run-example @@ -19,7 +19,7 @@ SCALA_VERSION=2.10 -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" export SPARK_HOME="$FWDIR" EXAMPLES_DIR="$FWDIR"/examples @@ -35,12 +35,12 @@ else fi if [ -f "$FWDIR/RELEASE" ]; then - export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` + export SPARK_EXAMPLES_JAR="`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar`" elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` + export SPARK_EXAMPLES_JAR="`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar`" fi -if [[ -z $SPARK_EXAMPLES_JAR ]]; then +if [[ -z "$SPARK_EXAMPLES_JAR" ]]; then echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 echo "You need to build Spark before running this program" 1>&2 exit 1 diff --git a/bin/spark-class b/bin/spark-class index c6543545a5e64..5f5f9ea74888d 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -27,12 +27,12 @@ esac SCALA_VERSION=2.10 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: spark-class []" 1>&2 @@ -105,7 +105,7 @@ else exit 1 fi fi -JAVA_VERSION=$($RUNNER -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size if [ "$JAVA_VERSION" -ge 18 ]; then @@ -117,7 +117,7 @@ JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" + JAVA_OPTS="$JAVA_OPTS `cat "$FWDIR"/conf/java-opts`" fi # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! @@ -126,21 +126,21 @@ TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar` + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar`" fi if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the Maven build # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`" fi # Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) +classpath_output=$("$FWDIR"/bin/compute-classpath.sh) if [[ "$?" != "0" ]]; then echo "$classpath_output" exit 1 else - CLASSPATH=$classpath_output + CLASSPATH="$classpath_output" fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then @@ -153,9 +153,9 @@ if [[ "$1" =~ org.apache.spark.tools.* ]]; then fi if $cygwin; then - CLASSPATH=`cygpath -wp $CLASSPATH` + CLASSPATH="`cygpath -wp "$CLASSPATH"`" if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` + export SPARK_TOOLS_JAR="`cygpath -w "$SPARK_TOOLS_JAR"`" fi fi export CLASSPATH diff --git a/bin/spark-shell b/bin/spark-shell index 0ab4e14f5b744..4a0670fc6c8aa 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -29,11 +29,11 @@ esac set -o posix ## Global script variables -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage() { echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } @@ -41,7 +41,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" @@ -54,11 +54,11 @@ function main() { # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" fi } diff --git a/bin/spark-sql b/bin/spark-sql index 2a3cb31f58e8d..ae096530cad04 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -27,7 +27,7 @@ CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { echo "Usage: ./bin/spark-sql [options] [cli option]" @@ -38,10 +38,10 @@ function usage { pattern+="\|--help" pattern+="\|=======" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "CLI options:" - $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -49,7 +49,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 277c4ce571ca2..c557311b4b20e 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,7 +19,7 @@ # NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! -export SPARK_HOME="$(cd `dirname $0`/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" ORIG_ARGS=("$@") while (($#)); do @@ -59,5 +59,5 @@ if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FI fi fi -exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" +exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/dev/check-license b/dev/check-license index 558e038afc01a..9ff0929e9a5e8 100755 --- a/dev/check-license +++ b/dev/check-license @@ -23,16 +23,16 @@ acquire_rat_jar () { URL1="http://search.maven.org/remotecontent?filepath=org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" URL2="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" - JAR=$rat_jar + JAR="$rat_jar" if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet if [ ! -f "$JAR" ]; then # Download printf "Attempting to fetch rat\n" - JAR_DL=${JAR}.part + JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent ${URL1} > "$JAR_DL" || curl --silent ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" + (curl --silent "${URL1}" > "$JAR_DL" || curl --silent "${URL2}" > "$JAR_DL") && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then (wget --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else @@ -50,7 +50,7 @@ acquire_rat_jar () { } # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then @@ -60,17 +60,17 @@ else fi export RAT_VERSION=0.10 -export rat_jar=$FWDIR/lib/apache-rat-${RAT_VERSION}.jar -mkdir -p $FWDIR/lib +export rat_jar="$FWDIR"/lib/apache-rat-${RAT_VERSION}.jar +mkdir -p "$FWDIR"/lib [[ -f "$rat_jar" ]] || acquire_rat_jar || { echo "Download failed. Obtain the rat jar manually and place it at $rat_jar" exit 1 } -$java_cmd -jar $rat_jar -E $FWDIR/.rat-excludes -d $FWDIR > rat-results.txt +$java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt -ERRORS=$(cat rat-results.txt | grep -e "??") +ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then echo "Could not find Apache license headers in the following files:" diff --git a/dev/lint-python b/dev/lint-python index 79bf70f0b8b13..772f856154ae0 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -18,10 +18,10 @@ # SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" -SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" -cd $SPARK_ROOT_DIR +cd "$SPARK_ROOT_DIR" # Get pep8 at runtime so that we don't rely on it being installed on the build server. #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 @@ -45,7 +45,7 @@ fi #+ first, but we do so so that the check status can #+ be output before the report, like with the #+ scalastyle and RAT checks. -python $PEP8_SCRIPT_PATH $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" +python "$PEP8_SCRIPT_PATH" $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" pep8_status=${PIPESTATUS[0]} #$? if [ $pep8_status -ne 0 ]; then diff --git a/dev/mima b/dev/mima index 09e4482af5f3d..f9b9b03538f15 100755 --- a/dev/mima +++ b/dev/mima @@ -21,12 +21,12 @@ set -o pipefail set -e # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update -export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"` +export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore diff --git a/dev/run-tests b/dev/run-tests index 90a8ce16f0f06..49a88085c80f7 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -18,7 +18,7 @@ # # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then diff --git a/dev/scalastyle b/dev/scalastyle index eb9b467965636..efb5f291ea3b7 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -19,7 +19,7 @@ echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt # Check style with YARN built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ diff --git a/make-distribution.sh b/make-distribution.sh index 14aed4a4b655b..9b012b9222db4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -28,7 +28,7 @@ set -o pipefail set -e # Figure out where the Spark framework is installed -FWDIR="$(cd `dirname $0`; pwd)" +FWDIR="$(cd "`dirname "$0"`"; pwd)" DISTDIR="$FWDIR/dist" SPARK_TACHYON=false diff --git a/python/run-tests b/python/run-tests index f2a80b4f1838b..226e9e2c3770a 100755 --- a/python/run-tests +++ b/python/run-tests @@ -19,7 +19,7 @@ # Figure out where the Spark framework is installed -FWDIR="$(cd `dirname $0`; cd ../; pwd)" +FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" # CD into the python directory to find things on the right path cd "$FWDIR/python" @@ -33,7 +33,9 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log + + SPARK_TESTING=1 "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log + FAILED=$((PIPESTATUS[0]||$FAILED)) # Fail and exit on the first test failure. diff --git a/sbin/slaves.sh b/sbin/slaves.sh index f89547fef9e46..1d4dc5edf9858 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -36,29 +36,29 @@ if [ $# -le 0 ]; then exit 1 fi -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" # If the slaves file is specified in the command line, # then it takes precedence over the definition in # spark-env.sh. Save it here. -HOSTLIST=$SPARK_SLAVES +HOSTLIST="$SPARK_SLAVES" # Check if --config is passed as an argument. It is an optional parameter. # Exit if the argument is not a directory. if [ "$1" == "--config" ] then shift - conf_dir=$1 + conf_dir="$1" if [ ! -d "$conf_dir" ] then echo "ERROR : $conf_dir is not a directory" echo $usage exit 1 else - export SPARK_CONF_DIR=$conf_dir + export SPARK_CONF_DIR="$conf_dir" fi shift fi @@ -79,7 +79,7 @@ if [ "$SPARK_SSH_OPTS" = "" ]; then fi for slave in `cat "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do - ssh $SPARK_SSH_OPTS $slave $"${@// /\\ }" \ + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ 2>&1 | sed "s/^/$slave: /" & if [ "$SPARK_SLAVE_SLEEP" != "" ]; then sleep $SPARK_SLAVE_SLEEP diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 5c87da5815b64..2718d6cba1c9a 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -21,19 +21,19 @@ # resolve links - $0 may be a softlink this="${BASH_SOURCE-$0}" -common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P) +common_bin="$(cd -P -- "$(dirname -- "$this")" && pwd -P)" script="$(basename -- "$this")" this="$common_bin/$script" # convert relative path to absolute path -config_bin=`dirname "$this"` -script=`basename "$this"` -config_bin=`cd "$config_bin"; pwd` +config_bin="`dirname "$this"`" +script="`basename "$this"`" +config_bin="`cd "$config_bin"; pwd`" this="$config_bin/$script" -export SPARK_PREFIX=`dirname "$this"`/.. -export SPARK_HOME=${SPARK_PREFIX} +export SPARK_PREFIX="`dirname "$this"`"/.. +export SPARK_HOME="${SPARK_PREFIX}" export SPARK_CONF_DIR="$SPARK_HOME/conf" # Add the PySpark classes to the PYTHONPATH: -export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$SPARK_HOME/python:$PYTHONPATH" +export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 9032f23ea8eff..bd476b400e1c3 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -37,8 +37,8 @@ if [ $# -le 1 ]; then exit 1 fi -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -50,14 +50,14 @@ sbin=`cd "$sbin"; pwd` if [ "$1" == "--config" ] then shift - conf_dir=$1 + conf_dir="$1" if [ ! -d "$conf_dir" ] then echo "ERROR : $conf_dir is not a directory" echo $usage exit 1 else - export SPARK_CONF_DIR=$conf_dir + export SPARK_CONF_DIR="$conf_dir" fi shift fi @@ -100,12 +100,12 @@ if [ "$SPARK_LOG_DIR" = "" ]; then export SPARK_LOG_DIR="$SPARK_HOME/logs" fi mkdir -p "$SPARK_LOG_DIR" -touch $SPARK_LOG_DIR/.spark_test > /dev/null 2>&1 +touch "$SPARK_LOG_DIR"/.spark_test > /dev/null 2>&1 TEST_LOG_DIR=$? if [ "${TEST_LOG_DIR}" = "0" ]; then - rm -f $SPARK_LOG_DIR/.spark_test + rm -f "$SPARK_LOG_DIR"/.spark_test else - chown $SPARK_IDENT_STRING $SPARK_LOG_DIR + chown "$SPARK_IDENT_STRING" "$SPARK_LOG_DIR" fi if [ "$SPARK_PID_DIR" = "" ]; then @@ -113,8 +113,8 @@ if [ "$SPARK_PID_DIR" = "" ]; then fi # some variables -log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out -pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid +log="$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out" +pid="$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid" # Set default scheduling priority if [ "$SPARK_NICENESS" = "" ]; then @@ -136,7 +136,7 @@ case $startStop in fi if [ "$SPARK_MASTER" != "" ]; then - echo rsync from $SPARK_MASTER + echo rsync from "$SPARK_MASTER" rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" fi diff --git a/sbin/spark-executor b/sbin/spark-executor index 3621321a9bc8d..674ce906d9421 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -17,10 +17,10 @@ # limitations under the License. # -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -export PYTHONPATH=$FWDIR/python:$PYTHONPATH -export PYTHONPATH=$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$FWDIR/python:$PYTHONPATH" +export PYTHONPATH="$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend +exec "$FWDIR"/bin/spark-class org.apache.spark.executor.MesosExecutorBackend diff --git a/sbin/start-all.sh b/sbin/start-all.sh index 5c89ab4d86b3a..1baf57cea09ee 100755 --- a/sbin/start-all.sh +++ b/sbin/start-all.sh @@ -21,8 +21,8 @@ # Starts the master on this node. # Starts a worker on each node specified in conf/slaves -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" TACHYON_STR="" diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 580ab471b8a79..7172ad15d88fc 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -24,8 +24,8 @@ # Use the SPARK_HISTORY_OPTS environment variable to set history server configuration. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" . "$SPARK_PREFIX/bin/load-spark-env.sh" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index c5c02491f78e1..17fff58f4f768 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -19,8 +19,8 @@ # Starts the master on the machine this script is executed on. -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" START_TACHYON=false diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index b563400dc24f3..2fc35309f4ca5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -20,7 +20,7 @@ # Usage: start-slave.sh # where is like "spark://localhost:7077" -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 4912d0c0c7dfd..ba1a84abc1fef 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" START_TACHYON=false @@ -46,11 +46,11 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_IP" = "" ]; then - SPARK_MASTER_IP=`hostname` + SPARK_MASTER_IP="`hostname`" fi if [ "$START_TACHYON" == "true" ]; then - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf $SPARK_MASTER_IP + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf "$SPARK_MASTER_IP" # set -t so we can call sudo SPARK_SSH_OPTS="-o StrictHostKeyChecking=no -t" "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/../tachyon/bin/tachyon-start.sh" worker SudoMount \; sleep 1 @@ -58,12 +58,12 @@ fi # Launch the slaves if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT + exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" else if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then SPARK_WORKER_WEBUI_PORT=8081 fi for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) done fi diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index c519a77df4a14..4ce40fe750384 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -24,7 +24,7 @@ set -o posix # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" CLASS_NOT_FOUND_EXIT_STATUS=1 @@ -38,10 +38,10 @@ function usage { pattern+="\|=======" pattern+="\|--help" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "Thrift server options:" - $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -49,7 +49,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 60b358d374565..298c6a9859795 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -21,8 +21,8 @@ # Run this on the master nde -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" # Load the Spark configuration . "$sbin/spark-config.sh" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh index c0034ad641cbe..6e6056359510f 100755 --- a/sbin/stop-history-server.sh +++ b/sbin/stop-history-server.sh @@ -19,7 +19,7 @@ # Stops the history server on the machine this script is executed on. -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 diff --git a/sbt/sbt b/sbt/sbt index 1b1aa1483a829..c172fa74bc771 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -3,32 +3,32 @@ # When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so # that we can run Hive to generate the golden answer. This is not required for normal development # or testing. -for i in $HIVE_HOME/lib/* -do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +for i in "$HIVE_HOME"/lib/* +do HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$i" done export HADOOP_CLASSPATH realpath () { ( - TARGET_FILE=$1 + TARGET_FILE="$1" - cd $(dirname $TARGET_FILE) - TARGET_FILE=$(basename $TARGET_FILE) + cd "$(dirname "$TARGET_FILE")" + TARGET_FILE="$(basename "$TARGET_FILE")" COUNT=0 while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ] do - TARGET_FILE=$(readlink $TARGET_FILE) - cd $(dirname $TARGET_FILE) - TARGET_FILE=$(basename $TARGET_FILE) + TARGET_FILE="$(readlink "$TARGET_FILE")" + cd $(dirname "$TARGET_FILE") + TARGET_FILE="$(basename $TARGET_FILE)" COUNT=$(($COUNT + 1)) done - echo $(pwd -P)/$TARGET_FILE + echo "$(pwd -P)/"$TARGET_FILE"" ) } -. $(dirname $(realpath $0))/sbt-launch-lib.bash +. "$(dirname "$(realpath "$0")")"/sbt-launch-lib.bash declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index fecc3d38a5fbd..7f05d2ef491a3 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -7,7 +7,7 @@ # TODO - Should we merge the main SBT script with this library? if test -z "$HOME"; then - declare -r script_dir="$(dirname $script_path)" + declare -r script_dir="$(dirname "$script_path")" else declare -r script_dir="$HOME/.sbt" fi @@ -46,20 +46,20 @@ acquire_sbt_jar () { if [[ ! -f "$sbt_jar" ]]; then # Download sbt launch jar if it hasn't been downloaded yet - if [ ! -f ${JAR} ]; then + if [ ! -f "${JAR}" ]; then # Download printf "Attempting to fetch sbt\n" - JAR_DL=${JAR}.part + JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent ${URL1} > ${JAR_DL} || curl --silent ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (curl --silent ${URL1} > "${JAR_DL}" || curl --silent ${URL2} > "${JAR_DL}") && mv "${JAR_DL}" "${JAR}" elif hash wget 2>/dev/null; then - (wget --quiet ${URL1} -O ${JAR_DL} || wget --quiet ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (wget --quiet ${URL1} -O "${JAR_DL}" || wget --quiet ${URL2} -O "${JAR_DL}") && mv "${JAR_DL}" "${JAR}" else printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" exit -1 fi fi - if [ ! -f ${JAR} ]; then + if [ ! -f "${JAR}" ]; then # We failed to download printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" exit -1 From 16a73c2473181e03d88001aa3e08e6ffac92eb8b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 8 Sep 2014 11:20:00 -0700 Subject: [PATCH 03/76] SPARK-2978. Transformation with MR shuffle semantics I didn't add this to the transformations list in the docs because it's kind of obscure, but would be happy to do so if others think it would be helpful. Author: Sandy Ryza Closes #2274 from sryza/sandy-spark-2978 and squashes the following commits: 4a5332a [Sandy Ryza] Fix Java test c04b447 [Sandy Ryza] Fix Python doc and add back deleted code 433ad5b [Sandy Ryza] Add Java test 4c25a54 [Sandy Ryza] Add s at the end and a couple other fixes 9b0ba99 [Sandy Ryza] Fix compilation 36e0571 [Sandy Ryza] Fix import ordering 48c12c2 [Sandy Ryza] Add Java version and additional doc e5381cd [Sandy Ryza] Fix python style warnings f147634 [Sandy Ryza] SPARK-2978. Transformation with MR shuffle semantics --- .../apache/spark/api/java/JavaPairRDD.scala | 26 ++++++++++++++++ .../spark/rdd/OrderedRDDFunctions.scala | 14 ++++++++- .../java/org/apache/spark/JavaAPISuite.java | 30 +++++++++++++++++++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 14 +++++++++ python/pyspark/rdd.py | 24 +++++++++++++++ python/pyspark/tests.py | 8 +++++ 6 files changed, 115 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index feeb6c02caa78..880f61c49726e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -758,6 +758,32 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) rdd.saveAsHadoopDataset(conf) } + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner): JavaPairRDD[K, V] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]] + repartitionAndSortWithinPartitions(partitioner, comp) + } + + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner, comp: Comparator[K]) + : JavaPairRDD[K, V] = { + implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering. + fromRDD( + new OrderedRDDFunctions[K, V, (K, V)](rdd).repartitionAndSortWithinPartitions(partitioner)) + } + /** * Sort the RDD by key, so that each partition contains a sorted range of the elements in * ascending order. Calling `collect` or `save` on the resulting RDD will return or output an diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index e98bad2026e32..d0dbfef35d03c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, RangePartitioner} +import org.apache.spark.{Logging, Partitioner, RangePartitioner} import org.apache.spark.annotation.DeveloperApi /** @@ -64,4 +64,16 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, new ShuffledRDD[K, V, V](self, part) .setKeyOrdering(if (ascending) ordering else ordering.reverse) } + + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = { + new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering) + } + } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e1c13de04a0be..be99dc501c4b2 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -189,6 +189,36 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @Test + public void repartitionAndSortWithinPartitions() { + List> pairs = new ArrayList>(); + pairs.add(new Tuple2(0, 5)); + pairs.add(new Tuple2(3, 8)); + pairs.add(new Tuple2(2, 6)); + pairs.add(new Tuple2(0, 8)); + pairs.add(new Tuple2(3, 8)); + pairs.add(new Tuple2(1, 3)); + + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + Partitioner partitioner = new Partitioner() { + public int numPartitions() { + return 2; + } + public int getPartition(Object key) { + return ((Integer)key).intValue() % 2; + } + }; + + JavaPairRDD repartitioned = + rdd.repartitionAndSortWithinPartitions(partitioner); + List>> partitions = repartitioned.glom().collect(); + Assert.assertEquals(partitions.get(0), Arrays.asList(new Tuple2(0, 5), + new Tuple2(0, 8), new Tuple2(2, 6))); + Assert.assertEquals(partitions.get(1), Arrays.asList(new Tuple2(1, 3), + new Tuple2(3, 8), new Tuple2(3, 8))); + } + @Test public void emptyRDD() { JavaRDD rdd = sc.emptyRDD(); diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 499dcda3dae8f..c1b501a75c8b8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -682,6 +682,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(data.sortBy(parse, true, 2)(NameOrdering, classTag[Person]).collect() === nameOrdered) } + test("repartitionAndSortWithinPartitions") { + val data = sc.parallelize(Seq((0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)), 2) + + val partitioner = new Partitioner { + def numPartitions: Int = 2 + def getPartition(key: Any): Int = key.asInstanceOf[Int] % 2 + } + + val repartitioned = data.repartitionAndSortWithinPartitions(partitioner) + val partitions = repartitioned.glom().collect() + assert(partitions(0) === Seq((0, 5), (0, 8), (2, 6))) + assert(partitions(1) === Seq((1, 3), (3, 8), (3, 8))) + } + test("intersection") { val all = sc.parallelize(1 to 10) val evens = sc.parallelize(2 to 10 by 2) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 266090e3ae8f3..5667154cb84a8 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -520,6 +520,30 @@ def __add__(self, other): raise TypeError return self.union(other) + def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=portable_hash, + ascending=True, keyfunc=lambda x: x): + """ + Repartition the RDD according to the given partitioner and, within each resulting partition, + sort records by their keys. + + >>> rdd = sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)]) + >>> rdd2 = rdd.repartitionAndSortWithinPartitions(2, lambda x: x % 2, 2) + >>> rdd2.glom().collect() + [[(0, 5), (0, 8), (2, 6)], [(1, 3), (3, 8), (3, 8)]] + """ + if numPartitions is None: + numPartitions = self._defaultReducePartitions() + + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == "true") + memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + serializer = self._jrdd_deserializer + + def sortPartition(iterator): + sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted + return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + + return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) + def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): """ Sorts this RDD, which is assumed to consist of (key, value) pairs. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 9fbeb36f4f1dd..0bd2a9e6c507d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -545,6 +545,14 @@ def test_histogram(self): self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) + def test_repartitionAndSortWithinPartitions(self): + rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2) + + repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) + partitions = repartitioned.glom().collect() + self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + class TestSQL(PySparkTestCase): From 386bc24ebe3e75875b9647d9223c62d7b9dc9963 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Mon, 8 Sep 2014 12:37:52 -0700 Subject: [PATCH 04/76] Provide a default PYSPARK_PYTHON for python/run_tests Without this the version of python used in the test is not recorded. The error is, Testing with Python version: ./run-tests: line 57: --version: command not found Author: Matthew Farrellee Closes #2300 from mattf/master-fix-python-run-tests and squashes the following commits: 65a09f5 [Matthew Farrellee] Provide a default PYSPARK_PYTHON for python/run_tests --- python/run-tests | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/run-tests b/python/run-tests index 226e9e2c3770a..d98840de59d2c 100755 --- a/python/run-tests +++ b/python/run-tests @@ -50,6 +50,8 @@ function run_test() { echo "Running PySpark tests. Output is in python/unit-tests.log." +export PYSPARK_PYTHON="python" + # Try to test with Python 2.6, since that's the minimum version that we support: if [ $(which python2.6) ]; then export PYSPARK_PYTHON="python2.6" From 26bc7655de18ab0191ded3f75cb77bc756dc1c03 Mon Sep 17 00:00:00 2001 From: Henry Cook Date: Mon, 8 Sep 2014 14:56:37 -0700 Subject: [PATCH 05/76] [SQL] Minor edits to sql programming guide. Author: Henry Cook Closes #2316 from hcook/sql-docs and squashes the following commits: 373f94b [Henry Cook] Minor edits to sql programming guide. --- docs/sql-programming-guide.md | 92 ++++++++++++++++++----------------- 1 file changed, 47 insertions(+), 45 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 1814fef465cac..d83efa4bab324 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -13,10 +13,10 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed -[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of +[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`. @@ -26,10 +26,10 @@ All of the examples on this page use sample data included in the Spark distribut
Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed -[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with +[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed of +[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects, along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table -in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
@@ -37,10 +37,10 @@ file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive]( Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed -[Row](api/python/pyspark.sql.Row-class.html) objects along with +[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed of +[Row](api/python/pyspark.sql.Row-class.html) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell. @@ -68,11 +68,11 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD {% endhighlight %} -In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict -super set of the functionality provided by the basic SQLContext. Additional features include +In addition to the basic SQLContext, you can also create a HiveContext, which provides a +superset of the functionality provided by the basic SQLContext. Additional features include the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing hive setup, and all of the data sources available to a SQLContext are still available. +existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to @@ -95,7 +95,7 @@ In addition to the basic SQLContext, you can also create a HiveContext, which pr super set of the functionality provided by the basic SQLContext. Additional features include the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing hive setup, and all of the data sources available to a SQLContext are still available. +existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to @@ -118,7 +118,7 @@ In addition to the basic SQLContext, you can also create a HiveContext, which pr super set of the functionality provided by the basic SQLContext. Additional features include the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing hive setup, and all of the data sources available to a SQLContext are still available. +existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to @@ -146,11 +146,11 @@ describes the various methods for loading data into a SchemaRDD. Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection based approach leads to more concise code and works well went the schema is known ahead -of time, while you are writing your Spark application. +reflection based approach leads to more concise code and works well when you already know the schema +while writing your Spark application. The second method for creating SchemaRDDs is through a programmatic interface that allows you to -construct a schema and then apply it to and existing RDD. While this method is more verbose, it allows +construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows you to construct SchemaRDDs when the columns and their types are not known until runtime. ### Inferring the Schema Using Reflection @@ -266,10 +266,10 @@ List teenagerNames = teenagers.map(new Function() {
-Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes . Rows are constructed by passing a list of -key/value pairs as kwargs to the Row class. The keys of this list define the columns names of the table, +Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes. Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, and the types are inferred by looking at the first row. Since we currently only look at the first -row, it is important that there is no missing data in the first row of the RDD. In future version we +row, it is important that there is no missing data in the first row of the RDD. In future versions we plan to more completely infer the schema by looking at more data, similar to the inference that is performed on JSON files. @@ -306,14 +306,14 @@ for teenName in teenNames.collect():
-In cases that case classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string or a text dataset will be parsed +When case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in the step 1. +`Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `applySchema` method provided by `SQLContext`. @@ -358,14 +358,14 @@ results.map(t => "Name: " + t(0)).collect().foreach(println)
-In cases that JavaBean classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string or a text dataset will be parsed and +When JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in the step 1. +`Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `applySchema` method provided by `JavaSQLContext`. @@ -427,10 +427,10 @@ List names = results.map(new Function() {
-For some cases (for example, the structure of records is encoded in a string or -a text dataset will be parsed and fields will be projected differently for -different users), it is desired to create `SchemaRDD` with a programmatically way. -It can be done with three steps. +When a dictionary of kwargs cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of tuples or lists from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of @@ -566,7 +566,7 @@ for teenName in teenNames.collect(): ### Configuration -Configuration of parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -575,8 +575,8 @@ Configuration of parquet can be done using the `setConf` method on SQLContext or @@ -584,14 +584,14 @@ Configuration of parquet can be done using the `setConf` method on SQLContext or @@ -805,9 +805,8 @@ Spark SQL can cache tables using an in-memory columnar format by calling `cacheT Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. -Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in -in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to -cache tables. +Note that if you call `cache` rather than `cacheTable`, tables will _not_ be cached using +the in-memory columnar format, and therefore `cacheTable` is strongly recommended for this use case. Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -833,7 +832,7 @@ Configuration of in-memory caching can be done using the `setConf` method on SQL
spark.sql.parquet.binaryAsString false - Some other parquet producing systems, in particular Impala and older versions of Spark SQL, do - not differentiate between binary data and strings when writing out the parquet schema. This + Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems.
spark.sql.parquet.cacheMetadata false - Turns on caching of parquet schema metadata. Can speed up querying + Turns on caching of Parquet schema metadata. Can speed up querying of static data.
spark.sql.parquet.compression.codec snappy - Sets the compression codec use when writing parquet files. Acceptable values include: + Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo.
-## Other Configuration +## Other Configuration Options The following options can also be used to tune the performance of query execution. It is possible that these options will be deprecated in future release as more optimizations are performed automatically. @@ -842,7 +841,7 @@ that these options will be deprecated in future release as more optimizations ar Property NameDefaultMeaning spark.sql.autoBroadcastJoinThreshold - false + 10000 Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently @@ -876,7 +875,7 @@ code. ## Running the Thrift JDBC server The Thrift JDBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 0.12. You can test the JDBC server with the beeline script comes with either Spark or Hive 0.12. +in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12. To start the JDBC server, run the following in the Spark directory: @@ -899,12 +898,12 @@ your machine and a blank password. For secure mode, please follow the instructio Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may also use the beeline script comes with Hive. +You may also use the beeline script that comes with Hive. ## Running the Spark SQL CLI The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. +queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. To start the Spark SQL CLI, run the following in the Spark directory: @@ -916,7 +915,10 @@ options. # Compatibility with Other Systems -## Migration Guide for Shark Users +## Migration Guide for Shark User + +### Scheduling +s To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: @@ -925,7 +927,7 @@ users can set the `spark.sql.thriftserver.scheduler.pool` variable: ### Reducer number In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark -SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value +SQL deprecates this property in favor of `spark.sql.shuffle.partitions`, whose default value is 200. Users may customize this property via `SET`: SET spark.sql.shuffle.partitions=10; From 939a322c85956eda150b10afb2ed1d8d959a7bdf Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 8 Sep 2014 15:45:28 -0700 Subject: [PATCH 06/76] [SPARK-3417] Use new-style classes in PySpark Tiny PR making SQLContext a new-style class. This allows various type logic to work more effectively ```Python In [1]: import pyspark In [2]: pyspark.sql.SQLContext.mro() Out[2]: [pyspark.sql.SQLContext, object] ``` Author: Matthew Rocklin Closes #2288 from mrocklin/sqlcontext-new-style-class and squashes the following commits: 4aadab6 [Matthew Rocklin] update other old-style classes a2dc02f [Matthew Rocklin] pyspark.sql.SQLContext is new-style class --- python/pyspark/mllib/random.py | 2 +- python/pyspark/mllib/util.py | 2 +- python/pyspark/sql.py | 2 +- python/pyspark/storagelevel.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 3e59c73db85e3..d53c95fd59c25 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -28,7 +28,7 @@ __all__ = ['RandomRDDs', ] -class RandomRDDs: +class RandomRDDs(object): """ Generator methods for creating RDDs comprised of i.i.d samples from some distribution. diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 4962d05491c03..1c7b8c809ab5b 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -25,7 +25,7 @@ from pyspark.serializers import NoOpSerializer -class MLUtils: +class MLUtils(object): """ Helper methods to load, save and pre-process data used in MLlib. diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 004d4937cbe1c..53eea6d6cf3ba 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -899,7 +899,7 @@ def __reduce__(self): return Row -class SQLContext: +class SQLContext(object): """Main entry point for Spark SQL functionality. diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 2aa0fb9d2c1ed..676aa0f7144aa 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -18,7 +18,7 @@ __all__ = ["StorageLevel"] -class StorageLevel: +class StorageLevel(object): """ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, From 08ce18881e09c6e91db9c410d1d9ce1e5ae63a62 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 8 Sep 2014 15:59:20 -0700 Subject: [PATCH 07/76] [SPARK-3019] Pluggable block transfer interface (BlockTransferService) This pull request creates a new BlockTransferService interface for block fetch/upload and refactors the existing ConnectionManager to implement BlockTransferService (NioBlockTransferService). Most of the changes are simply moving code around. The main class to inspect is ShuffleBlockFetcherIterator. Review guide: - Most of the ConnectionManager code is now in network.cm package - ManagedBuffer is a new buffer abstraction backed by several different implementations (file segment, nio ByteBuffer, Netty ByteBuf) - BlockTransferService is the main internal interface introduced in this PR - NioBlockTransferService implements BlockTransferService and replaces the old BlockManagerWorker - ShuffleBlockFetcherIterator replaces the told BlockFetcherIterator to use the new interface TODOs that should be separate PRs: - Implement NettyBlockTransferService - Finalize the API/semantics for ManagedBuffer.release() Author: Reynold Xin Closes #2240 from rxin/blockTransferService and squashes the following commits: 64cd9d7 [Reynold Xin] Merge branch 'master' into blockTransferService 1dfd3d7 [Reynold Xin] Limit the length of the FileInputStream. 1332156 [Reynold Xin] Fixed style violation from refactoring. 2960c93 [Reynold Xin] Added ShuffleBlockFetcherIteratorSuite. e29c721 [Reynold Xin] Updated comment for ShuffleBlockFetcherIterator. 8a1046e [Reynold Xin] Code review feedback: 2c6b1e1 [Reynold Xin] Removed println in test cases. 2a907e4 [Reynold Xin] Merge branch 'master' into blockTransferService-merge 07ccf0d [Reynold Xin] Added init check to CMBlockTransferService. 98c668a [Reynold Xin] Added failure handling and fixed unit tests. ae05fcd [Reynold Xin] Updated tests, although DistributedSuite is hanging. d8d595c [Reynold Xin] Merge branch 'master' of github.com:apache/spark into blockTransferService 9ef279c [Reynold Xin] Initial refactoring to move ConnectionManager to use the BlockTransferService. --- .../scala/org/apache/spark/SparkEnv.scala | 15 +- ...eiverTest.scala => BlockDataManager.scala} | 29 +- .../spark/network/BlockFetchingListener.scala | 37 +++ .../spark/network/BlockTransferService.scala | 131 +++++++++ .../spark/network/ConnectionManagerTest.scala | 103 ------- .../apache/spark/network/ManagedBuffer.scala | 107 +++++++ .../org/apache/spark/network/SenderTest.scala | 76 ----- .../nio}/BlockMessage.scala | 24 +- .../nio}/BlockMessageArray.scala | 12 +- .../network/{ => nio}/BufferMessage.scala | 5 +- .../spark/network/{ => nio}/Connection.scala | 10 +- .../network/{ => nio}/ConnectionId.scala | 6 +- .../network/{ => nio}/ConnectionManager.scala | 23 +- .../{ => nio}/ConnectionManagerId.scala | 6 +- .../spark/network/{ => nio}/Message.scala | 7 +- .../network/{ => nio}/MessageChunk.scala | 4 +- .../{ => nio}/MessageChunkHeader.scala | 9 +- .../network/nio/NioBlockTransferService.scala | 205 +++++++++++++ .../network/{ => nio}/SecurityMessage.scala | 10 +- .../spark/serializer/KryoSerializer.scala | 2 +- .../shuffle/FileShuffleBlockManager.scala | 35 ++- .../shuffle/IndexShuffleBlockManager.scala | 24 +- .../spark/shuffle/ShuffleBlockManager.scala | 6 +- .../hash/BlockStoreShuffleFetcher.scala | 14 +- .../shuffle/hash/HashShuffleReader.scala | 4 +- .../spark/storage/BlockFetcherIterator.scala | 254 ---------------- .../apache/spark/storage/BlockManager.scala | 98 +++---- .../apache/spark/storage/BlockManagerId.scala | 4 +- .../spark/storage/BlockManagerWorker.scala | 147 ---------- .../storage/ShuffleBlockFetcherIterator.scala | 271 ++++++++++++++++++ .../apache/spark/storage/ThreadingTest.scala | 120 -------- .../org/apache/spark/DistributedSuite.scala | 15 +- .../{ => nio}/ConnectionManagerSuite.scala | 17 +- .../hash/HashShuffleManagerSuite.scala | 17 +- .../storage/BlockFetcherIteratorSuite.scala | 237 --------------- .../spark/storage/BlockManagerSuite.scala | 133 +-------- .../spark/storage/DiskBlockManagerSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 183 ++++++++++++ 38 files changed, 1129 insertions(+), 1273 deletions(-) rename core/src/main/scala/org/apache/spark/network/{ReceiverTest.scala => BlockDataManager.scala} (56%) create mode 100644 core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala create mode 100644 core/src/main/scala/org/apache/spark/network/BlockTransferService.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala create mode 100644 core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/SenderTest.scala rename core/src/main/scala/org/apache/spark/{storage => network/nio}/BlockMessage.scala (89%) rename core/src/main/scala/org/apache/spark/{storage => network/nio}/BlockMessageArray.scala (97%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/BufferMessage.scala (98%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/Connection.scala (99%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionId.scala (88%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionManager.scala (98%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionManagerId.scala (88%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/Message.scala (95%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/MessageChunk.scala (96%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/MessageChunkHeader.scala (93%) create mode 100644 core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala rename core/src/main/scala/org/apache/spark/network/{ => nio}/SecurityMessage.scala (95%) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala rename core/src/test/scala/org/apache/spark/network/{ => nio}/ConnectionManagerSuite.scala (97%) delete mode 100644 core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 20a7444cfc5ee..dd95e406f2a8e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -31,7 +31,8 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.ConnectionManager +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} @@ -59,8 +60,8 @@ class SparkEnv ( val mapOutputTracker: MapOutputTracker, val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, + val blockTransferService: BlockTransferService, val blockManager: BlockManager, - val connectionManager: ConnectionManager, val securityManager: SecurityManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, @@ -88,6 +89,8 @@ class SparkEnv ( // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. // actorSystem.awaitTermination() + + // Note that blockTransferService is stopped by BlockManager since it is started by it. } private[spark] @@ -223,14 +226,14 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) + val blockTransferService = new NioBlockTransferService(conf, securityManager) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, mapOutputTracker, shuffleManager) - - val connectionManager = blockManager.connectionManager + serializer, conf, mapOutputTracker, shuffleManager, blockTransferService) val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) @@ -278,8 +281,8 @@ object SparkEnv extends Logging { mapOutputTracker, shuffleManager, broadcastManager, + blockTransferService, blockManager, - connectionManager, securityManager, httpFileServer, sparkFilesDir, diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala similarity index 56% rename from core/src/main/scala/org/apache/spark/network/ReceiverTest.scala rename to core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 53a6038a9b59e..e0e91724271c8 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -17,21 +17,20 @@ package org.apache.spark.network -import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.storage.StorageLevel -private[spark] object ReceiverTest { - def main(args: Array[String]) { - val conf = new SparkConf - val manager = new ConnectionManager(9999, conf, new SecurityManager(conf)) - println("Started connection manager with id = " + manager.id) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) - Some(Message.createBufferMessage(buffer, msg.id)) - }) - Thread.currentThread.join() - } -} +trait BlockDataManager { + + /** + * Interface to get local block data. + * + * @return Some(buffer) if the block exists locally, and None if it doesn't. + */ + def getBlockData(blockId: String): Option[ManagedBuffer] + /** + * Put the block locally, using the given storage level. + */ + def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala new file mode 100644 index 0000000000000..34acaa563ca58 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala @@ -0,0 +1,37 @@ +/* + * 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.network + +import java.util.EventListener + + +/** + * Listener callback interface for [[BlockTransferService.fetchBlocks]]. + */ +trait BlockFetchingListener extends EventListener { + + /** + * Called once per successfully fetched block. + */ + def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit + + /** + * Called upon failures. For each failure, this is called only once (i.e. not once per block). + */ + def onBlockFetchFailure(exception: Throwable): Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala new file mode 100644 index 0000000000000..84d991fa6808c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -0,0 +1,131 @@ +/* + * 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.network + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.Duration + +import org.apache.spark.storage.StorageLevel + + +abstract class BlockTransferService { + + /** + * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch + * local blocks or put local blocks. + */ + def init(blockDataManager: BlockDataManager) + + /** + * Tear down the transfer service. + */ + def stop(): Unit + + /** + * Port number the service is listening on, available only after [[init]] is invoked. + */ + def port: Int + + /** + * Host name the service is listening on, available only after [[init]] is invoked. + */ + def hostName: String + + /** + * Fetch a sequence of blocks from a remote node asynchronously, + * available only after [[init]] is invoked. + * + * Note that [[BlockFetchingListener.onBlockFetchSuccess]] is called once per block, + * while [[BlockFetchingListener.onBlockFetchFailure]] is called once per failure (not per block). + * + * Note that this API takes a sequence so the implementation can batch requests, and does not + * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as + * the data of a block is fetched, rather than waiting for all blocks to be fetched. + */ + def fetchBlocks( + hostName: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + */ + def uploadBlock( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel): Future[Unit] + + /** + * A special case of [[fetchBlocks]], as it fetches only one block and is blocking. + * + * It is also only available after [[init]] is invoked. + */ + def fetchBlockSync(hostName: String, port: Int, blockId: String): ManagedBuffer = { + // A monitor for the thread to wait on. + val lock = new Object + @volatile var result: Either[ManagedBuffer, Throwable] = null + fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { + override def onBlockFetchFailure(exception: Throwable): Unit = { + lock.synchronized { + result = Right(exception) + lock.notify() + } + } + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + lock.synchronized { + result = Left(data) + lock.notify() + } + } + }) + + // Sleep until result is no longer null + lock.synchronized { + while (result == null) { + try { + lock.wait() + } catch { + case e: InterruptedException => + } + } + } + + result match { + case Left(data) => data + case Right(e) => throw e + } + } + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + * + * This method is similar to [[uploadBlock]], except this one blocks the thread + * until the upload finishes. + */ + def uploadBlockSync( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel): Unit = { + Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala deleted file mode 100644 index 4894ecd41f6eb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ /dev/null @@ -1,103 +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.network - -import java.nio.ByteBuffer - -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.io.Source - -import org.apache.spark._ - -private[spark] object ConnectionManagerTest extends Logging{ - def main(args: Array[String]) { - // - the master URL - a list slaves to run connectionTest on - // [num of tasks] - the number of parallel tasks to be initiated default is number of slave - // hosts [size of msg in MB (integer)] - the size of messages to be sent in each task, - // default is 10 [count] - how many times to run, default is 3 [await time in seconds] : - // await time (in seconds), default is 600 - if (args.length < 2) { - println("Usage: ConnectionManagerTest [num of tasks] " + - "[size of msg in MB (integer)] [count] [await time in seconds)] ") - System.exit(1) - } - - if (args(0).startsWith("local")) { - println("This runs only on a mesos cluster") - } - - val sc = new SparkContext(args(0), "ConnectionManagerTest") - val slavesFile = Source.fromFile(args(1)) - val slaves = slavesFile.mkString.split("\n") - slavesFile.close() - - /* println("Slaves") */ - /* slaves.foreach(println) */ - val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 - val count = if (args.length > 4) args(4).toInt else 3 - val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second - println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + - "msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) - val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( - i => SparkEnv.get.connectionManager.id).collect() - println("\nSlave ConnectionManagerIds") - slaveConnManagerIds.foreach(println) - println - - (0 until count).foreach(i => { - val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { - val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - logInfo("Received [" + msg + "] from [" + id + "]") - None - }) - - val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) - buffer.flip - - val startTime = System.currentTimeMillis - val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => - { - val bufferMessage = Message.createBufferMessage(buffer.duplicate) - logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") - connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) - } - } - val results = futures.map(f => Await.result(f, awaitTime)) - val finishTime = System.currentTimeMillis - Thread.sleep(5000) - - val mb = size * results.size / 1024.0 / 1024.0 - val ms = finishTime - startTime - val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * - 1000.0) + " MB/s" - logInfo(resultStr) - resultStr - }).collect() - - println("---------------------") - println("Run " + i) - resultStrs.foreach(println) - println("---------------------") - }) - } -} - diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala new file mode 100644 index 0000000000000..dcecb6beeea9b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -0,0 +1,107 @@ +/* + * 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.network + +import java.io.{FileInputStream, RandomAccessFile, File, InputStream} +import java.nio.ByteBuffer +import java.nio.channels.FileChannel.MapMode + +import com.google.common.io.ByteStreams +import io.netty.buffer.{ByteBufInputStream, ByteBuf} + +import org.apache.spark.util.ByteBufferInputStream + + +/** + * This interface provides an immutable view for data in the form of bytes. The implementation + * should specify how the data is provided: + * + * - FileSegmentManagedBuffer: data backed by part of a file + * - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer + * - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf + */ +sealed abstract class ManagedBuffer { + // Note that all the methods are defined with parenthesis because their implementations can + // have side effects (io operations). + + /** Number of bytes of the data. */ + def size: Long + + /** + * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the + * returned ByteBuffer should not affect the content of this buffer. + */ + def nioByteBuffer(): ByteBuffer + + /** + * Exposes this buffer's data as an InputStream. The underlying implementation does not + * necessarily check for the length of bytes read, so the caller is responsible for making sure + * it does not go over the limit. + */ + def inputStream(): InputStream +} + + +/** + * A [[ManagedBuffer]] backed by a segment in a file + */ +final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long) + extends ManagedBuffer { + + override def size: Long = length + + override def nioByteBuffer(): ByteBuffer = { + val channel = new RandomAccessFile(file, "r").getChannel + channel.map(MapMode.READ_ONLY, offset, length) + } + + override def inputStream(): InputStream = { + val is = new FileInputStream(file) + is.skip(offset) + ByteStreams.limit(is, length) + } +} + + +/** + * A [[ManagedBuffer]] backed by [[java.nio.ByteBuffer]]. + */ +final class NioByteBufferManagedBuffer(buf: ByteBuffer) extends ManagedBuffer { + + override def size: Long = buf.remaining() + + override def nioByteBuffer() = buf.duplicate() + + override def inputStream() = new ByteBufferInputStream(buf) +} + + +/** + * A [[ManagedBuffer]] backed by a Netty [[ByteBuf]]. + */ +final class NettyByteBufManagedBuffer(buf: ByteBuf) extends ManagedBuffer { + + override def size: Long = buf.readableBytes() + + override def nioByteBuffer() = buf.nioBuffer() + + override def inputStream() = new ByteBufInputStream(buf) + + // TODO(rxin): Promote this to top level ManagedBuffer interface and add documentation for it. + def release(): Unit = buf.release() +} diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala deleted file mode 100644 index ea2ad104ecae1..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ /dev/null @@ -1,76 +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.network - -import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import scala.util.Try - -private[spark] object SenderTest { - def main(args: Array[String]) { - - if (args.length < 2) { - println("Usage: SenderTest ") - System.exit(1) - } - - val targetHost = args(0) - val targetPort = args(1).toInt - val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val conf = new SparkConf - val manager = new ConnectionManager(0, conf, new SecurityManager(conf)) - println("Started connection manager with id = " + manager.id) - - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - println("Received [" + msg + "] from [" + id + "]") - None - }) - - val size = 100 * 1024 * 1024 - val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) - buffer.flip - - val targetServer = args(0) - - val count = 100 - (0 until count).foreach(i => { - val dataMessage = Message.createBufferMessage(buffer.duplicate) - val startTime = System.currentTimeMillis - /* println("Started timer at " + startTime) */ - val promise = manager.sendMessageReliably(targetConnectionManagerId, dataMessage) - val responseStr: String = Try(Await.result(promise, Duration.Inf)) - .map { response => - val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array, "utf-8") - }.getOrElse("none") - - val finishTime = System.currentTimeMillis - val mb = size / 1024.0 / 1024.0 - val ms = finishTime - startTime - // val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms - // * 1000.0) + " MB/s" - val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + - (mb / ms * 1000.0).toInt + "MB/s) | Response = " + responseStr - println(resultStr) - }) - } -} - diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala similarity index 89% rename from core/src/main/scala/org/apache/spark/storage/BlockMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala index a2bfce7b4a0fa..b573f1a8a5fcb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.StringBuilder +import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} -import org.apache.spark.network._ +import scala.collection.mutable.{ArrayBuffer, StringBuilder} +// private[spark] because we need to register them in Kryo private[spark] case class GetBlock(id: BlockId) private[spark] case class GotBlock(id: BlockId, data: ByteBuffer) private[spark] case class PutBlock(id: BlockId, data: ByteBuffer, level: StorageLevel) -private[spark] class BlockMessage() { +private[nio] class BlockMessage() { // Un-initialized: typ = 0 // GetBlock: typ = 1 // GotBlock: typ = 2 @@ -159,7 +159,7 @@ private[spark] class BlockMessage() { } } -private[spark] object BlockMessage { +private[nio] object BlockMessage { val TYPE_NON_INITIALIZED: Int = 0 val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 @@ -194,16 +194,4 @@ private[spark] object BlockMessage { newBlockMessage.set(putBlock) newBlockMessage } - - def main(args: Array[String]) { - val B = new BlockMessage() - val blockId = TestBlockId("ABC") - B.set(new PutBlock(blockId, ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2)) - val bMsg = B.toBufferMessage - val C = new BlockMessage() - C.set(bMsg) - - println(B.getId + " " + B.getLevel) - println(C.getId + " " + C.getLevel) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala rename to core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala index 973d85c0a9b3a..a1a2c00ed1542 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import org.apache.spark._ -import org.apache.spark.network._ +import org.apache.spark.storage.{StorageLevel, TestBlockId} + +import scala.collection.mutable.ArrayBuffer -private[spark] +private[nio] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { @@ -102,7 +102,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } } -private[spark] object BlockMessageArray { +private[nio] object BlockMessageArray { def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/network/BufferMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala index af35f1fc3e459..3b245c5c7a4f3 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer @@ -23,7 +23,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.storage.BlockManager -private[spark] + +private[nio] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) extends Message(Message.BUFFER_MESSAGE, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/network/Connection.scala rename to core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 5285ec82c1b64..74074a8dcbfff 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net._ import java.nio._ import java.nio.channels._ -import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} - import org.apache.spark._ -private[spark] +import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} + +private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId) extends Logging { @@ -190,7 +190,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, } -private[spark] +private[nio] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, remoteId_ : ConnectionManagerId, id_ : ConnectionId) extends Connection(SocketChannel.open, selector_, remoteId_, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/network/ConnectionId.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala index d579c165a1917..764dc5e5503ed 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio -private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { +private[nio] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } -private[spark] object ConnectionId { +private[nio] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/network/ConnectionManager.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 578d806263006..09d3ea306515b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -15,32 +15,27 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.io.IOException +import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ -import java.net._ -import java.util.{Timer, TimerTask} import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{LinkedBlockingDeque, ThreadPoolExecutor, TimeUnit} +import java.util.{Timer, TimerTask} -import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.collection.mutable.SynchronizedMap -import scala.collection.mutable.SynchronizedQueue - -import scala.concurrent.{Await, ExecutionContext, Future, Promise} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, SynchronizedMap, SynchronizedQueue} import scala.concurrent.duration._ +import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} -private[spark] class ConnectionManager( + +private[nio] class ConnectionManager( port: Int, conf: SparkConf, securityManager: SecurityManager, @@ -904,7 +899,7 @@ private[spark] class ConnectionManager( private[spark] object ConnectionManager { - import ExecutionContext.Implicits.global + import scala.concurrent.ExecutionContext.Implicits.global def main(args: Array[String]) { val conf = new SparkConf diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala index 57f7586883af1..cbb37ec5ced1f 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net.InetSocketAddress import org.apache.spark.util.Utils -private[spark] case class ConnectionManagerId(host: String, port: Int) { +private[nio] case class ConnectionManagerId(host: String, port: Int) { // DEBUG code Utils.checkHost(host) assert (port > 0) @@ -30,7 +30,7 @@ private[spark] case class ConnectionManagerId(host: String, port: Int) { } -private[spark] object ConnectionManagerId { +private[nio] object ConnectionManagerId { def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { new ConnectionManagerId(socketAddress.getHostName, socketAddress.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/network/Message.scala rename to core/src/main/scala/org/apache/spark/network/nio/Message.scala index 04ea50f62918c..0b874c2891255 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -15,14 +15,15 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net.InetSocketAddress import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -private[spark] abstract class Message(val typ: Long, val id: Int) { + +private[nio] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null var started = false var startTime = -1L @@ -42,7 +43,7 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { } -private[spark] object Message { +private[nio] object Message { val BUFFER_MESSAGE = 1111111111L var lastId = 1 diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/network/MessageChunk.scala rename to core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala index d0f986a12bfe0..278c5ac356ef2 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -private[network] +private[nio] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { val size = if (buffer == null) 0 else buffer.remaining diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala rename to core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala index f3ecca5f992e0..6e20f291c5cec 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio -import java.net.InetAddress -import java.net.InetSocketAddress +import java.net.{InetAddress, InetSocketAddress} import java.nio.ByteBuffer -private[spark] class MessageChunkHeader( +private[nio] class MessageChunkHeader( val typ: Long, val id: Int, val totalSize: Int, @@ -57,7 +56,7 @@ private[spark] class MessageChunkHeader( } -private[spark] object MessageChunkHeader { +private[nio] object MessageChunkHeader { val HEADER_SIZE = 45 def create(buffer: ByteBuffer): MessageChunkHeader = { diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala new file mode 100644 index 0000000000000..59958ee894230 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -0,0 +1,205 @@ +/* + * 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.network.nio + +import java.nio.ByteBuffer + +import scala.concurrent.Future + +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.network._ +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils + + +/** + * A [[BlockTransferService]] implementation based on [[ConnectionManager]], a custom + * implementation using Java NIO. + */ +final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityManager) + extends BlockTransferService with Logging { + + private var cm: ConnectionManager = _ + + private var blockDataManager: BlockDataManager = _ + + /** + * Port number the service is listening on, available only after [[init]] is invoked. + */ + override def port: Int = { + checkInit() + cm.id.port + } + + /** + * Host name the service is listening on, available only after [[init]] is invoked. + */ + override def hostName: String = { + checkInit() + cm.id.host + } + + /** + * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch + * local blocks or put local blocks. + */ + override def init(blockDataManager: BlockDataManager): Unit = { + this.blockDataManager = blockDataManager + cm = new ConnectionManager( + conf.getInt("spark.blockManager.port", 0), + conf, + securityManager, + "Connection manager for block manager") + cm.onReceiveMessage(onBlockMessageReceive) + } + + /** + * Tear down the transfer service. + */ + override def stop(): Unit = { + if (cm != null) { + cm.stop() + } + } + + override def fetchBlocks( + hostName: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit = { + checkInit() + + val cmId = new ConnectionManagerId(hostName, port) + val blockMessageArray = new BlockMessageArray(blockIds.map { blockId => + BlockMessage.fromGetBlock(GetBlock(BlockId(blockId))) + }) + + val future = cm.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + + // Register the listener on success/failure future callback. + future.onSuccess { case message => + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + listener.onBlockFetchFailure( + new SparkException(s"Unexpected message ${blockMessage.getType} received from $cmId")) + } else { + val blockId = blockMessage.getId + val networkSize = blockMessage.getData.limit() + listener.onBlockFetchSuccess( + blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + } + } + }(cm.futureExecContext) + + future.onFailure { case exception => + listener.onBlockFetchFailure(exception) + }(cm.futureExecContext) + } + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + * + * This call blocks until the upload completes, or throws an exception upon failures. + */ + override def uploadBlock( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel) + : Future[Unit] = { + checkInit() + val msg = PutBlock(BlockId(blockId), blockData.nioByteBuffer(), level) + val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg)) + val remoteCmId = new ConnectionManagerId(hostName, port) + val reply = cm.sendMessageReliably(remoteCmId, blockMessageArray.toBufferMessage) + reply.map(x => ())(cm.futureExecContext) + } + + private def checkInit(): Unit = if (cm == null) { + throw new IllegalStateException(getClass.getName + " has not been initialized") + } + + private def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { + logDebug("Handling message " + msg) + msg match { + case bufferMessage: BufferMessage => + try { + logDebug("Handling as a buffer message " + bufferMessage) + val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) + logDebug("Parsed as a block message array") + val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) + Some(new BlockMessageArray(responseMessages).toBufferMessage) + } catch { + case e: Exception => { + logError("Exception handling buffer message", e) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } + } + + case otherMessage: Any => + logError("Unknown type message received: " + otherMessage) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } + } + + private def processBlockMessage(blockMessage: BlockMessage): Option[BlockMessage] = { + blockMessage.getType match { + case BlockMessage.TYPE_PUT_BLOCK => + val msg = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) + logDebug("Received [" + msg + "]") + putBlock(msg.id.toString, msg.data, msg.level) + None + + case BlockMessage.TYPE_GET_BLOCK => + val msg = new GetBlock(blockMessage.getId) + logDebug("Received [" + msg + "]") + val buffer = getBlock(msg.id.toString) + if (buffer == null) { + return None + } + Some(BlockMessage.fromGotBlock(GotBlock(msg.id, buffer))) + + case _ => None + } + } + + private def putBlock(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + val startTimeMs = System.currentTimeMillis() + logDebug("PutBlock " + blockId + " started from " + startTimeMs + " with data: " + bytes) + blockDataManager.putBlockData(blockId, new NioByteBufferManagedBuffer(bytes), level) + logDebug("PutBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + + " with data size: " + bytes.limit) + } + + private def getBlock(blockId: String): ByteBuffer = { + val startTimeMs = System.currentTimeMillis() + logDebug("GetBlock " + blockId + " started from " + startTimeMs) + val buffer = blockDataManager.getBlockData(blockId).orNull + logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + + " and got buffer " + buffer) + buffer.nioByteBuffer() + } +} diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/network/SecurityMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala index 9af9e2e8e9e59..747a2088a7258 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.StringBuilder +import scala.collection.mutable.{ArrayBuffer, StringBuilder} import org.apache.spark._ -import org.apache.spark.network._ /** * SecurityMessage is class that contains the connectionId and sasl token @@ -54,7 +52,7 @@ import org.apache.spark.network._ * - Length of the token * - Token */ -private[spark] class SecurityMessage() extends Logging { +private[nio] class SecurityMessage extends Logging { private var connectionId: String = null private var token: Array[Byte] = null @@ -134,7 +132,7 @@ private[spark] class SecurityMessage() extends Logging { } } -private[spark] object SecurityMessage { +private[nio] object SecurityMessage { /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 87ef9bb0b43c6..d6386f8c06fff 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -27,9 +27,9 @@ import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast +import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ -import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.CompactBuffer diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 96facccd52373..439981d232349 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.{SparkEnv, SparkConf, Logging} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup import org.apache.spark.storage._ @@ -166,34 +167,30 @@ class FileShuffleBlockManager(conf: SparkConf) } } - /** - * Returns the physical file segment in which the given BlockId is located. - */ - private def getBlockLocation(id: ShuffleBlockId): FileSegment = { + override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + val segment = getBlockData(blockId) + Some(segment.nioByteBuffer()) + } + + override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. - val shuffleState = shuffleStates(id.shuffleId) + val shuffleState = shuffleStates(blockId.shuffleId) val iter = shuffleState.allFileGroups.iterator while (iter.hasNext) { - val segment = iter.next.getFileSegmentFor(id.mapId, id.reduceId) - if (segment.isDefined) { return segment.get } + val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) + if (segmentOpt.isDefined) { + val segment = segmentOpt.get + return new FileSegmentManagedBuffer(segment.file, segment.offset, segment.length) + } } - throw new IllegalStateException("Failed to find shuffle block: " + id) + throw new IllegalStateException("Failed to find shuffle block: " + blockId) } else { - val file = blockManager.diskBlockManager.getFile(id) - new FileSegment(file, 0, file.length()) + val file = blockManager.diskBlockManager.getFile(blockId) + new FileSegmentManagedBuffer(file, 0, file.length) } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockLocation(blockId) - blockManager.diskStore.getBytes(segment) - } - - override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = { - Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])) - } - /** Remove all the blocks / files and metadata related to a particular shuffle. */ def removeShuffle(shuffleId: ShuffleId): Boolean = { // Do not change the ordering of this, if shuffleStates should be removed only diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 8bb9efc46cc58..4ab34336d3f01 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkEnv +import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} import org.apache.spark.storage._ /** @@ -89,10 +90,11 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { } } - /** - * Get the location of a block in a map output file. Uses the index file we create for it. - * */ - private def getBlockLocation(blockId: ShuffleBlockId): FileSegment = { + override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + Some(getBlockData(blockId).nioByteBuffer()) + } + + override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) @@ -102,20 +104,14 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { in.skip(blockId.reduceId * 8) val offset = in.readLong() val nextOffset = in.readLong() - new FileSegment(getDataFile(blockId.shuffleId, blockId.mapId), offset, nextOffset - offset) + new FileSegmentManagedBuffer( + getDataFile(blockId.shuffleId, blockId.mapId), + offset, + nextOffset - offset) } finally { in.close() } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockLocation(blockId) - blockManager.diskStore.getBytes(segment) - } - - override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = { - Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])) - } - override def stop() = {} } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index 4240580250046..63863cc0250a3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -19,7 +19,8 @@ package org.apache.spark.shuffle import java.nio.ByteBuffer -import org.apache.spark.storage.{FileSegment, ShuffleBlockId} +import org.apache.spark.network.ManagedBuffer +import org.apache.spark.storage.ShuffleBlockId private[spark] trait ShuffleBlockManager { @@ -31,8 +32,7 @@ trait ShuffleBlockManager { */ def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] - def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] + def getBlockData(blockId: ShuffleBlockId): ManagedBuffer def stop(): Unit } - diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 12b475658e29d..6cf9305977a3c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -21,10 +21,9 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} +import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} import org.apache.spark.util.CompletionIterator private[hash] object BlockStoreShuffleFetcher extends Logging { @@ -32,8 +31,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer, - shuffleMetrics: ShuffleReadMetrics) + serializer: Serializer) : Iterator[T] = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) @@ -74,7 +72,13 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer, shuffleMetrics) + val blockFetcherItr = new ShuffleBlockFetcherIterator( + context, + SparkEnv.get.blockTransferService, + blockManager, + blocksByAddress, + serializer, + SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 7bed97a63f0f6..88a5f1e5ddf58 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -36,10 +36,8 @@ private[spark] class HashShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() val ser = Serializer.getSerializer(dep.serializer) - val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser, - readMetrics) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser) val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala deleted file mode 100644 index e35b7fe62c753..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ /dev/null @@ -1,254 +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.storage - -import java.util.concurrent.LinkedBlockingQueue -import org.apache.spark.network.netty.client.{BlockClientListener, LazyInitIterator, ReferenceCountedBuffer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet -import scala.collection.mutable.Queue -import scala.util.{Failure, Success} - -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.executor.ShuffleReadMetrics -import org.apache.spark.network.BufferMessage -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils - -/** - * A block fetcher iterator interface for fetching shuffle blocks. - */ -private[storage] -trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { - def initialize() -} - - -private[storage] -object BlockFetcherIterator { - - /** - * A request to fetch blocks from a remote BlockManager. - * @param address remote BlockManager to fetch from. - * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight. - */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { - val size = blocks.map(_._2).sum - } - - /** - * Result of a fetch from a remote block. A failure is represented as size == -1. - * @param blockId block id - * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. - * @param deserialize closure to return the result in the form of an Iterator. - */ - class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - - // TODO: Refactor this whole thing to make code more reusable. - class BasicBlockFetcherIterator( - private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, - readMetrics: ShuffleReadMetrics) - extends BlockFetcherIterator { - - import blockManager._ - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - - // Total number blocks fetched (local + remote). Also number of FetchResults expected - protected var _numBlocksToFetch = 0 - - protected var startTime = System.currentTimeMillis - - // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks - protected val localBlocksToFetch = new ArrayBuffer[BlockId]() - - // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks - protected val remoteBlocksToFetch = new HashSet[BlockId]() - - // A queue to hold our results. - protected val results = new LinkedBlockingQueue[FetchResult] - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - protected val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - protected var bytesInFlight = 0L - - protected def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) - val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onComplete { - case Success(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - val networkSize = blockMessage.getData.limit() - results.put(new FetchResult(blockId, sizeMap(blockId), - () => dataDeserialize(blockId, blockMessage.getData, serializer))) - // TODO: NettyBlockFetcherIterator has some race conditions where multiple threads can - // be incrementing bytes read at the same time (SPARK-2625). - readMetrics.remoteBytesRead += networkSize - readMetrics.remoteBlocksFetched += 1 - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case Failure(exception) => { - logError("Could not get block(s) from " + cmId, exception) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { - // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) - - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - var totalBlocks = 0 - for ((address, blockInfos) <- blocksByAddress) { - totalBlocks += blockInfos.size - if (address == blockManagerId) { - // Filter out zero-sized blocks - localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) - _numBlocksToFetch += localBlocksToFetch.size - } else { - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(BlockId, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - // Skip empty blocks - if (size > 0) { - curBlocks += ((blockId, size)) - remoteBlocksToFetch += blockId - _numBlocksToFetch += 1 - curRequestSize += size - } else if (size < 0) { - throw new BlockException(blockId, "Negative block size " + size) - } - if (curRequestSize >= targetRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curBlocks = new ArrayBuffer[(BlockId, Long)] - logDebug(s"Creating fetch request of $curRequestSize at $address") - curRequestSize = 0 - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + - totalBlocks + " blocks") - remoteRequests - } - - protected def getLocalBlocks() { - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlocksToFetch) { - try { - readMetrics.localBlocksFetched += 1 - results.put(new FetchResult(id, 0, () => getLocalShuffleFromDisk(id, serializer).get)) - logDebug("Got local block " + id) - } catch { - case e: Exception => { - logError(s"Error occurred while fetching local blocks", e) - results.put(new FetchResult(id, -1, null)) - return - } - } - } - } - - override def initialize() { - // Split local and remote blocks. - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numFetches = remoteRequests.size - fetchRequests.size - logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue - // as they arrive. - @volatile protected var resultsGotten = 0 - - override def hasNext: Boolean = resultsGotten < _numBlocksToFetch - - override def next(): (BlockId, Option[Iterator[Any]]) = { - resultsGotten += 1 - val startFetchWait = System.currentTimeMillis() - val result = results.take() - val stopFetchWait = System.currentTimeMillis() - readMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) - if (! result.failed) bytesInFlight -= result.size - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } - // End of BasicBlockFetcherIterator -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a714142763243..d1bee3d2c033c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -20,6 +20,8 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} +import scala.concurrent.ExecutionContext.Implicits.global + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ @@ -58,18 +60,14 @@ private[spark] class BlockManager( defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, - shuffleManager: ShuffleManager) - extends BlockDataProvider with Logging { + shuffleManager: ShuffleManager, + blockTransferService: BlockTransferService) + extends BlockDataManager with Logging { - private val port = conf.getInt("spark.blockManager.port", 0) + blockTransferService.init(this) val diskBlockManager = new DiskBlockManager(this, conf) - val connectionManager = - new ConnectionManager(port, conf, securityManager, "Connection manager for block manager") - - implicit val futureExecContext = connectionManager.futureExecContext private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -89,11 +87,7 @@ private[spark] class BlockManager( } val blockManagerId = BlockManagerId( - executorId, connectionManager.id.host, connectionManager.id.port) - - // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory - // for receiving shuffle outputs) - val maxBytesInFlight = conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 + executorId, blockTransferService.hostName, blockTransferService.port) // Whether to compress broadcast variables that are stored private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) @@ -136,11 +130,11 @@ private[spark] class BlockManager( master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, - securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, - shuffleManager: ShuffleManager) = { + shuffleManager: ShuffleManager, + blockTransferService: BlockTransferService) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager, mapOutputTracker, shuffleManager) + conf, mapOutputTracker, shuffleManager, blockTransferService) } /** @@ -149,7 +143,6 @@ private[spark] class BlockManager( */ private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) - BlockManagerWorker.startBlockManagerWorker(this) } /** @@ -212,20 +205,33 @@ private[spark] class BlockManager( } } - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + /** + * Interface to get local block data. + * + * @return Some(buffer) if the block exists locally, and None if it doesn't. + */ + override def getBlockData(blockId: String): Option[ManagedBuffer] = { val bid = BlockId(blockId) if (bid.isShuffle) { - shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId]) + Some(shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId])) } else { val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] if (blockBytesOpt.isDefined) { - Right(blockBytesOpt.get) + val buffer = blockBytesOpt.get + Some(new NioByteBufferManagedBuffer(buffer)) } else { - throw new BlockNotFoundException(blockId) + None } } } + /** + * Put the block locally, using the given storage level. + */ + override def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit = { + putBytes(BlockId(blockId), data.nioByteBuffer(), level) + } + /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -333,16 +339,10 @@ private[spark] class BlockManager( * shuffle blocks. It is safe to do so without a lock on block info since disk store * never deletes (recent) items. */ - def getLocalShuffleFromDisk( - blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - - val shuffleBlockManager = shuffleManager.shuffleBlockManager - val values = shuffleBlockManager.getBytes(blockId.asInstanceOf[ShuffleBlockId]).map( - bytes => this.dataDeserialize(blockId, bytes, serializer)) - - values.orElse { - throw new BlockException(blockId, s"Block $blockId not found on disk, though it should be") - } + def getLocalShuffleFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { + val buf = shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) + val is = wrapForCompression(blockId, buf.inputStream()) + Some(serializer.newInstance().deserializeStream(is).asIterator) } /** @@ -513,8 +513,9 @@ private[spark] class BlockManager( val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") - val data = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) + val data = blockTransferService.fetchBlockSync( + loc.host, loc.port, blockId.toString).nioByteBuffer() + if (data != null) { if (asBlockResult) { return Some(new BlockResult( @@ -548,22 +549,6 @@ private[spark] class BlockManager( None } - /** - * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns - * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined - * fashion as they're received. Expects a size in bytes to be provided for each block fetched, - * so that we can control the maxMegabytesInFlight for the fetch. - */ - def getMultiple( - blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, - readMetrics: ShuffleReadMetrics): BlockFetcherIterator = { - val iter = new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer, - readMetrics) - iter.initialize() - iter - } - def putIterator( blockId: BlockId, values: Iterator[Any], @@ -816,12 +801,15 @@ private[spark] class BlockManager( data.rewind() logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + s"To node: $peer") - val putBlock = PutBlock(blockId, data, tLevel) - val cmId = new ConnectionManagerId(peer.host, peer.port) - val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) - if (!syncPutBlockSuccess) { - logError(s"Failed to call syncPutBlock to $peer") + + try { + blockTransferService.uploadBlockSync( + peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) + } catch { + case e: Exception => + logError(s"Failed to replicate block to $peer", e) } + logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) } @@ -1051,7 +1039,7 @@ private[spark] class BlockManager( } def stop(): Unit = { - connectionManager.stop() + blockTransferService.stop() diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index b7bcb2d85d0ee..d4487fce49ab6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -36,8 +36,8 @@ import org.apache.spark.util.Utils class BlockManagerId private ( private var executorId_ : String, private var host_ : String, - private var port_ : Int - ) extends Externalizable { + private var port_ : Int) + extends Externalizable { private def this() = this(null, null, 0) // For deserialization only diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala deleted file mode 100644 index bf002a42d5dc5..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ /dev/null @@ -1,147 +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.storage - -import java.nio.ByteBuffer - -import org.apache.spark.Logging -import org.apache.spark.network._ -import org.apache.spark.util.Utils - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import scala.util.{Try, Failure, Success} - -/** - * A network interface for BlockManager. Each slave should have one - * BlockManagerWorker. - * - * TODO: Use event model. - */ -private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { - - blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) - - def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { - logDebug("Handling message " + msg) - msg match { - case bufferMessage: BufferMessage => { - try { - logDebug("Handling as a buffer message " + bufferMessage) - val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) - logDebug("Parsed as a block message array") - val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) - Some(new BlockMessageArray(responseMessages).toBufferMessage) - } catch { - case e: Exception => { - logError("Exception handling buffer message", e) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) - } - } - } - case otherMessage: Any => { - logError("Unknown type message received: " + otherMessage) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) - } - } - } - - def processBlockMessage(blockMessage: BlockMessage): Option[BlockMessage] = { - blockMessage.getType match { - case BlockMessage.TYPE_PUT_BLOCK => { - val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) - logDebug("Received [" + pB + "]") - putBlock(pB.id, pB.data, pB.level) - None - } - case BlockMessage.TYPE_GET_BLOCK => { - val gB = new GetBlock(blockMessage.getId) - logDebug("Received [" + gB + "]") - val buffer = getBlock(gB.id) - if (buffer == null) { - return None - } - Some(BlockMessage.fromGotBlock(GotBlock(gB.id, buffer))) - } - case _ => None - } - } - - private def putBlock(id: BlockId, bytes: ByteBuffer, level: StorageLevel) { - val startTimeMs = System.currentTimeMillis() - logDebug("PutBlock " + id + " started from " + startTimeMs + " with data: " + bytes) - blockManager.putBytes(id, bytes, level) - logDebug("PutBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " with data size: " + bytes.limit) - } - - private def getBlock(id: BlockId): ByteBuffer = { - val startTimeMs = System.currentTimeMillis() - logDebug("GetBlock " + id + " started from " + startTimeMs) - val buffer = blockManager.getLocalBytes(id) match { - case Some(bytes) => bytes - case None => null - } - logDebug("GetBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " and got buffer " + buffer) - buffer - } -} - -private[spark] object BlockManagerWorker extends Logging { - private var blockManagerWorker: BlockManagerWorker = null - - def startBlockManagerWorker(manager: BlockManager) { - blockManagerWorker = new BlockManagerWorker(manager) - } - - def syncPutBlock(msg: PutBlock, toConnManagerId: ConnectionManagerId): Boolean = { - val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val blockMessage = BlockMessage.fromPutBlock(msg) - val blockMessageArray = new BlockMessageArray(blockMessage) - val resultMessage = Try(Await.result(connectionManager.sendMessageReliably( - toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) - resultMessage.isSuccess - } - - def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { - val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val blockMessage = BlockMessage.fromGetBlock(msg) - val blockMessageArray = new BlockMessageArray(blockMessage) - val responseMessage = Try(Await.result(connectionManager.sendMessageReliably( - toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) - responseMessage match { - case Success(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - logDebug("Response message received " + bufferMessage) - BlockMessageArray.fromBufferMessage(bufferMessage).foreach(blockMessage => { - logDebug("Found " + blockMessage) - return blockMessage.getData - }) - } - case Failure(exception) => logDebug("No response message received") - } - null - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala new file mode 100644 index 0000000000000..c8e708aa6b1bc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -0,0 +1,271 @@ +/* + * 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.storage + +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet +import scala.collection.mutable.Queue + +import org.apache.spark.{TaskContext, Logging, SparkException} +import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService} +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils + + +/** + * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block + * manager. For remote blocks, it fetches them using the provided BlockTransferService. + * + * This creates an iterator of (BlockID, values) tuples so the caller can handle blocks in a + * pipelined fashion as they are received. + * + * The implementation throttles the remote fetches to they don't exceed maxBytesInFlight to avoid + * using too much memory. + * + * @param context [[TaskContext]], used for metrics update + * @param blockTransferService [[BlockTransferService]] for fetching remote blocks + * @param blockManager [[BlockManager]] for reading local blocks + * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. + * For each block we also require the size (in bytes as a long field) in + * order to throttle the memory usage. + * @param serializer serializer used to deserialize the data. + * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. + */ +private[spark] +final class ShuffleBlockFetcherIterator( + context: TaskContext, + blockTransferService: BlockTransferService, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer, + maxBytesInFlight: Long) + extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { + + import ShuffleBlockFetcherIterator._ + + /** + * Total number of blocks to fetch. This can be smaller than the total number of blocks + * in [[blocksByAddress]] because we filter out zero-sized blocks in [[initialize]]. + * + * This should equal localBlocks.size + remoteBlocks.size. + */ + private[this] var numBlocksToFetch = 0 + + /** + * The number of blocks proccessed by the caller. The iterator is exhausted when + * [[numBlocksProcessed]] == [[numBlocksToFetch]]. + */ + private[this] var numBlocksProcessed = 0 + + private[this] val startTime = System.currentTimeMillis + + /** Local blocks to fetch, excluding zero-sized blocks. */ + private[this] val localBlocks = new ArrayBuffer[BlockId]() + + /** Remote blocks to fetch, excluding zero-sized blocks. */ + private[this] val remoteBlocks = new HashSet[BlockId]() + + /** + * A queue to hold our results. This turns the asynchronous model provided by + * [[BlockTransferService]] into a synchronous model (iterator). + */ + private[this] val results = new LinkedBlockingQueue[FetchResult] + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + private[this] val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + private[this] var bytesInFlight = 0L + + private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + + initialize() + + private[this] def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + bytesInFlight += req.size + + // so we can look up the size of each blockID + val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val blockIds = req.blocks.map(_._1.toString) + + blockTransferService.fetchBlocks(req.address.host, req.address.port, blockIds, + new BlockFetchingListener { + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), + () => serializer.newInstance().deserializeStream( + blockManager.wrapForCompression(BlockId(blockId), data.inputStream())).asIterator + )) + shuffleMetrics.remoteBytesRead += data.size + shuffleMetrics.remoteBlocksFetched += 1 + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + + override def onBlockFetchFailure(e: Throwable): Unit = { + logError("Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + // Note that there is a chance that some blocks have been fetched successfully, but we + // still add them to the failed queue. This is fine because when the caller see a + // FetchFailedException, it is going to fail the entire task anyway. + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + ) + } + + private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + + // Tracks total number of blocks (including zero sized blocks) + var totalBlocks = 0 + for ((address, blockInfos) <- blocksByAddress) { + totalBlocks += blockInfos.size + if (address == blockManager.blockManagerId) { + // Filter out zero-sized blocks + localBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) + numBlocksToFetch += localBlocks.size + } else { + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(BlockId, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + // Skip empty blocks + if (size > 0) { + curBlocks += ((blockId, size)) + remoteBlocks += blockId + numBlocksToFetch += 1 + curRequestSize += size + } else if (size < 0) { + throw new BlockException(blockId, "Negative block size " + size) + } + if (curRequestSize >= targetRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") + curRequestSize = 0 + } + } + // Add in the final request + if (curBlocks.nonEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo(s"Getting $numBlocksToFetch non-empty blocks out of $totalBlocks blocks") + remoteRequests + } + + private[this] def fetchLocalBlocks() { + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlocks) { + try { + shuffleMetrics.localBlocksFetched += 1 + results.put(new FetchResult( + id, 0, () => blockManager.getLocalShuffleFromDisk(id, serializer).get)) + logDebug("Got local block " + id) + } catch { + case e: Exception => + logError(s"Error occurred while fetching local blocks", e) + results.put(new FetchResult(id, -1, null)) + return + } + } + } + + private[this] def initialize(): Unit = { + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + fetchLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch + + override def next(): (BlockId, Option[Iterator[Any]]) = { + numBlocksProcessed += 1 + val startFetchWait = System.currentTimeMillis() + val result = results.take() + val stopFetchWait = System.currentTimeMillis() + shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) + if (!result.failed) { + bytesInFlight -= result.size + } + // Send fetch requests up to maxBytesInFlight + while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } +} + + +private[storage] +object ShuffleBlockFetcherIterator { + + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of tuple, where the first element is the block id, + * and the second element is the estimated size, used to calculate bytesInFlight. + */ + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { + val size = blocks.map(_._2).sum + } + + /** + * Result of a fetch from a remote block. A failure is represented as size == -1. + * @param blockId block id + * @param size estimated size of the block, used to calculate bytesInFlight. + * Note that this is NOT the exact bytes. + * @param deserialize closure to return the result in the form of an Iterator. + */ + class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala deleted file mode 100644 index 7540f0d5e2a5a..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import java.util.concurrent.ArrayBlockingQueue - -import akka.actor._ -import org.apache.spark.shuffle.hash.HashShuffleManager -import util.Random - -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} -import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.KryoSerializer - -/** - * This class tests the BlockManager and MemoryStore for thread safety and - * deadlocks. It spawns a number of producer and consumer threads. Producer - * threads continuously pushes blocks into the BlockManager and consumer - * threads continuously retrieves the blocks form the BlockManager and tests - * whether the block is correct or not. - */ -private[spark] object ThreadingTest { - - val numProducers = 5 - val numBlocksPerProducer = 20000 - - private[spark] class ProducerThread(manager: BlockManager, id: Int) extends Thread { - val queue = new ArrayBlockingQueue[(BlockId, Seq[Int])](100) - - override def run() { - for (i <- 1 to numBlocksPerProducer) { - val blockId = TestBlockId("b-" + id + "-" + i) - val blockSize = Random.nextInt(1000) - val block = (1 to blockSize).map(_ => Random.nextInt()) - val level = randomLevel() - val startTime = System.currentTimeMillis() - manager.putIterator(blockId, block.iterator, level, tellMaster = true) - println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") - queue.add((blockId, block)) - } - println("Producer thread " + id + " terminated") - } - - def randomLevel(): StorageLevel = { - math.abs(Random.nextInt()) % 4 match { - case 0 => StorageLevel.MEMORY_ONLY - case 1 => StorageLevel.MEMORY_ONLY_SER - case 2 => StorageLevel.MEMORY_AND_DISK - case 3 => StorageLevel.MEMORY_AND_DISK_SER - } - } - } - - private[spark] class ConsumerThread( - manager: BlockManager, - queue: ArrayBlockingQueue[(BlockId, Seq[Int])] - ) extends Thread { - var numBlockConsumed = 0 - - override def run() { - println("Consumer thread started") - while(numBlockConsumed < numBlocksPerProducer) { - val (blockId, block) = queue.take() - val startTime = System.currentTimeMillis() - manager.get(blockId) match { - case Some(retrievedBlock) => - assert(retrievedBlock.data.toList.asInstanceOf[List[Int]] == block.toList, - "Block " + blockId + " did not match") - println("Got block " + blockId + " in " + - (System.currentTimeMillis - startTime) + " ms") - case None => - assert(false, "Block " + blockId + " could not be retrieved") - } - numBlockConsumed += 1 - } - println("Consumer thread terminated") - } - } - - def main(args: Array[String]) { - System.setProperty("spark.kryoserializer.buffer.mb", "1") - val actorSystem = ActorSystem("test") - val conf = new SparkConf() - val serializer = new KryoSerializer(conf) - val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) - val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) - val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) - val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) - producers.foreach(_.start) - consumers.foreach(_.start) - producers.foreach(_.join) - consumers.foreach(_.join) - blockManager.stop() - blockManagerMaster.stop() - actorSystem.shutdown() - actorSystem.awaitTermination() - println("Everything stopped.") - println( - "It will take sometime for the JVM to clean all temporary files and shutdown. Sit tight.") - } -} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 41c294f727b3c..81b64c36ddca1 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -24,8 +24,7 @@ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} import org.apache.spark.SparkContext._ -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.storage.{BlockManagerWorker, GetBlock, RDDBlockId, StorageLevel} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} @@ -136,7 +135,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter sc.parallelize(1 to 10, 2).foreach { x => if (x == 1) System.exit(42) } } assert(thrown.getClass === classOf[SparkException]) - System.out.println(thrown.getMessage) assert(thrown.getMessage.contains("failed 4 times")) } } @@ -202,12 +200,13 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager - blockManager.master.getLocations(blockId).foreach(id => { - val bytes = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(id.host, id.port)) - val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList + val blockTransfer = SparkEnv.get.blockTransferService + blockManager.master.getLocations(blockId).foreach { cmId => + val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, blockId.toString) + val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) + .asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) - }) + } } test("compute without caching when no partitions fit in memory") { diff --git a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala rename to core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index e2f4d4c57cdb5..9f49587cdc670 100644 --- a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -15,23 +15,18 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.io.IOException import java.nio._ -import java.util.concurrent.TimeoutException -import org.apache.spark.{SecurityManager, SparkConf} -import org.scalatest.FunSuite - -import org.mockito.Mockito._ -import org.mockito.Matchers._ - -import scala.concurrent.TimeoutException -import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ +import scala.concurrent.{Await, TimeoutException} import scala.language.postfixOps -import scala.util.{Failure, Success, Try} + +import org.scalatest.FunSuite + +import org.apache.spark.{SecurityManager, SparkConf} /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 6061e544e79b4..ba47fe5e25b9b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkEnv, SparkContext, LocalSparkContext, SparkConf} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.FileShuffleBlockManager import org.apache.spark.storage.{ShuffleBlockId, FileSegment} @@ -32,10 +33,12 @@ import org.apache.spark.storage.{ShuffleBlockId, FileSegment} class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { private val testConf = new SparkConf(false) - private def checkSegments(segment1: FileSegment, segment2: FileSegment) { - assert (segment1.file.getCanonicalPath === segment2.file.getCanonicalPath) - assert (segment1.offset === segment2.offset) - assert (segment1.length === segment2.length) + private def checkSegments(expected: FileSegment, buffer: ManagedBuffer) { + assert(buffer.isInstanceOf[FileSegmentManagedBuffer]) + val segment = buffer.asInstanceOf[FileSegmentManagedBuffer] + assert(expected.file.getCanonicalPath === segment.file.getCanonicalPath) + assert(expected.offset === segment.offset) + assert(expected.length === segment.length) } test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { @@ -95,14 +98,12 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { writer.commitAndClose() } // check before we register. - checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0)).left.get) + checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0))) shuffle3.releaseWriters(success = true) - checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0)).left.get) + checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0))) shuffleBlockManager.removeShuffle(1) - } - def writeToFile(file: File, numBytes: Int) { val writer = new FileWriter(file, true) for (i <- 0 until numBytes) writer.write(i) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala deleted file mode 100644 index 3c86f6bafcaa3..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala +++ /dev/null @@ -1,237 +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.storage - -import java.io.IOException -import java.nio.ByteBuffer - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.future -import scala.concurrent.ExecutionContext.Implicits.global - -import org.scalatest.{FunSuite, Matchers} - -import org.mockito.Mockito._ -import org.mockito.Matchers.{any, eq => meq} -import org.mockito.stubbing.Answer -import org.mockito.invocation.InvocationOnMock - -import org.apache.spark.storage.BlockFetcherIterator._ -import org.apache.spark.network.{ConnectionManager, Message} -import org.apache.spark.executor.ShuffleReadMetrics - -class BlockFetcherIteratorSuite extends FunSuite with Matchers { - - test("block fetch from local fails using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - doReturn(connManager).when(blockManager).connectionManager - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - val answer = new Answer[Option[Iterator[Any]]] { - override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { - throw new Exception - } - } - - // 3rd block is going to fail - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) - - val bmId = BlockManagerId("test-client", "test-client", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, - new ShuffleReadMetrics()) - - iterator.initialize() - - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - // the 2nd element of the tuple returned by iterator.next should be defined when fetching successfully - assert(iterator.next()._2.isDefined, "1st element should be defined but is not actually defined") - verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, "2nd element should be defined but is not actually defined") - verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - // 3rd fetch should be failed - intercept[Exception] { - iterator.next() - } - verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) - } - - - test("block fetch from local succeed using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - doReturn(connManager).when(blockManager).connectionManager - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - - // All blocks should be fetched successfully - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) - - val bmId = BlockManagerId("test-client", "test-client", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, - new ShuffleReadMetrics()) - - iterator.initialize() - - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 1st element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next._2.isDefined, "All elements should be defined but 2nd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 3rd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 4th element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 5th element is not actually defined") - - verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) - } - - test("block fetch from remote fails using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - when(blockManager.connectionManager).thenReturn(connManager) - - val f = future { - throw new IOException("Send failed or we received an error ACK") - } - when(connManager.sendMessageReliably(any(), - any())).thenReturn(f) - when(blockManager.futureExecContext).thenReturn(global) - - when(blockManager.blockManagerId).thenReturn( - BlockManagerId("test-client", "test-client", 1)) - when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) - - val blId1 = ShuffleBlockId(0,0,0) - val blId2 = ShuffleBlockId(0,1,0) - val bmId = BlockManagerId("test-server", "test-server", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L))) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null, new ShuffleReadMetrics()) - - iterator.initialize() - iterator.foreach{ - case (_, r) => { - (!r.isDefined) should be(true) - } - } - } - - test("block fetch from remote succeed using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - when(blockManager.connectionManager).thenReturn(connManager) - - val blId1 = ShuffleBlockId(0,0,0) - val blId2 = ShuffleBlockId(0,1,0) - val buf1 = ByteBuffer.allocate(4) - val buf2 = ByteBuffer.allocate(4) - buf1.putInt(1) - buf1.flip() - buf2.putInt(1) - buf2.flip() - val blockMessage1 = BlockMessage.fromGotBlock(GotBlock(blId1, buf1)) - val blockMessage2 = BlockMessage.fromGotBlock(GotBlock(blId2, buf2)) - val blockMessageArray = new BlockMessageArray( - Seq(blockMessage1, blockMessage2)) - - val bufferMessage = blockMessageArray.toBufferMessage - val buffer = ByteBuffer.allocate(bufferMessage.size) - val arrayBuffer = new ArrayBuffer[ByteBuffer] - bufferMessage.buffers.foreach{ b => - buffer.put(b) - } - buffer.flip() - arrayBuffer += buffer - - val f = future { - Message.createBufferMessage(arrayBuffer) - } - when(connManager.sendMessageReliably(any(), - any())).thenReturn(f) - when(blockManager.futureExecContext).thenReturn(global) - - when(blockManager.blockManagerId).thenReturn( - BlockManagerId("test-client", "test-client", 1)) - when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) - - val bmId = BlockManagerId("test-server", "test-server", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L))) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null, new ShuffleReadMetrics()) - iterator.initialize() - iterator.foreach{ - case (_, r) => { - (r.isDefined) should be(true) - } - } - } -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c200654162268..e251660dae5de 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,15 +21,19 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit +import org.apache.spark.network.nio.NioBlockTransferService + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + import akka.actor._ import akka.pattern.ask import akka.util.Timeout -import org.apache.spark.shuffle.hash.HashShuffleManager -import org.mockito.invocation.InvocationOnMock -import org.mockito.Matchers.any -import org.mockito.Mockito.{doAnswer, mock, spy, when} -import org.mockito.stubbing.Answer +import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ @@ -38,18 +42,12 @@ import org.scalatest.Matchers import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod -import org.apache.spark.network.{Message, ConnectionManagerId} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.language.implicitConversions -import scala.language.postfixOps -import org.apache.spark.shuffle.ShuffleBlockManager class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { @@ -74,8 +72,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { - new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, - mapOutputTracker, shuffleManager) + val transfer = new NioBlockTransferService(conf, securityMgr) + new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + mapOutputTracker, shuffleManager, transfer) } before { @@ -793,8 +792,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. + val transfer = new NioBlockTransferService(conf, securityMgr) store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) + mapOutputTracker, shuffleManager, transfer) // The put should fail since a1 is not serializable. class UnserializableClass @@ -1005,109 +1005,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } - test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) - - val worker = spy(new BlockManagerWorker(store)) - val connManagerId = mock(classOf[ConnectionManagerId]) - - // setup request block messages - val reqBlId1 = ShuffleBlockId(0,0,0) - val reqBlId2 = ShuffleBlockId(0,1,0) - val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) - val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) - val reqBlockMessages = new BlockMessageArray( - Seq(reqBlockMessage1, reqBlockMessage2)) - val reqBufferMessage = reqBlockMessages.toBufferMessage - - val answer = new Answer[Option[BlockMessage]] { - override def answer(invocation: InvocationOnMock) - :Option[BlockMessage]= { - throw new Exception - } - } - - doAnswer(answer).when(worker).processBlockMessage(any()) - - // Test when exception was thrown during processing block messages - var ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) - - assert(ackMessage.isDefined, "When Exception was thrown in " + - "BlockManagerWorker#processBlockMessage, " + - "ackMessage should be defined") - assert(ackMessage.get.hasError, "When Exception was thown in " + - "BlockManagerWorker#processBlockMessage, " + - "ackMessage should have error") - - val notBufferMessage = mock(classOf[Message]) - - // Test when not BufferMessage was received - ackMessage = worker.onBlockMessageReceive(notBufferMessage, connManagerId) - assert(ackMessage.isDefined, "When not BufferMessage was passed to " + - "BlockManagerWorker#onBlockMessageReceive, " + - "ackMessage should be defined") - assert(ackMessage.get.hasError, "When not BufferMessage was passed to " + - "BlockManagerWorker#onBlockMessageReceive, " + - "ackMessage should have error") - } - - test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) - - val worker = spy(new BlockManagerWorker(store)) - val connManagerId = mock(classOf[ConnectionManagerId]) - - // setup request block messages - val reqBlId1 = ShuffleBlockId(0,0,0) - val reqBlId2 = ShuffleBlockId(0,1,0) - val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) - val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) - val reqBlockMessages = new BlockMessageArray( - Seq(reqBlockMessage1, reqBlockMessage2)) - - val tmpBufferMessage = reqBlockMessages.toBufferMessage - val buffer = ByteBuffer.allocate(tmpBufferMessage.size) - val arrayBuffer = new ArrayBuffer[ByteBuffer] - tmpBufferMessage.buffers.foreach{ b => - buffer.put(b) - } - buffer.flip() - arrayBuffer += buffer - val reqBufferMessage = Message.createBufferMessage(arrayBuffer) - - // setup ack block messages - val buf1 = ByteBuffer.allocate(4) - val buf2 = ByteBuffer.allocate(4) - buf1.putInt(1) - buf1.flip() - buf2.putInt(1) - buf2.flip() - val ackBlockMessage1 = BlockMessage.fromGotBlock(GotBlock(reqBlId1, buf1)) - val ackBlockMessage2 = BlockMessage.fromGotBlock(GotBlock(reqBlId2, buf2)) - - val answer = new Answer[Option[BlockMessage]] { - override def answer(invocation: InvocationOnMock) - :Option[BlockMessage]= { - if (invocation.getArguments()(0).asInstanceOf[BlockMessage].eq( - reqBlockMessage1)) { - return Some(ackBlockMessage1) - } else { - return Some(ackBlockMessage2) - } - } - } - - doAnswer(answer).when(worker).processBlockMessage(any()) - - val ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) - assert(ackMessage.isDefined, "When BlockManagerWorker#onBlockMessageReceive " + - "was executed successfully, ackMessage should be defined") - assert(!ackMessage.get.hasError, "When BlockManagerWorker#onBlockMessageReceive " + - "was executed successfully, ackMessage should not have error") - } - test("reserve/release unroll memory") { store = makeBlockManager(12000) val memoryStore = store.memoryStore diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 26082ded8ca7a..e4522e00a622d 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.{File, FileWriter} +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.shuffle.hash.HashShuffleManager import scala.collection.mutable @@ -52,7 +53,6 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before rootDir1 = Files.createTempDir() rootDir1.deleteOnExit() rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath - println("Created root dirs: " + rootDirs) } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala new file mode 100644 index 0000000000000..809bd70929656 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -0,0 +1,183 @@ +/* + * 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.storage + +import org.apache.spark.TaskContext +import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} + +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer + +import org.scalatest.FunSuite + + +class ShuffleBlockFetcherIteratorSuite extends FunSuite { + + test("handle local read failures in BlockManager") { + val transfer = mock(classOf[BlockTransferService]) + val blockManager = mock(classOf[BlockManager]) + doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + val answer = new Answer[Option[Iterator[Any]]] { + override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { + throw new Exception + } + } + + // 3rd block is going to fail + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) + doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + // Without exhausting the iterator, the iterator should be lazy and not call + // getLocalShuffleFromDisk. + verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + // the 2nd element of the tuple returned by iterator.next should be defined when + // fetching successfully + assert(iterator.next()._2.isDefined, + "1st element should be defined but is not actually defined") + verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next()._2.isDefined, + "2nd element should be defined but is not actually defined") + verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + // 3rd fetch should be failed + intercept[Exception] { + iterator.next() + } + verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) + } + + test("handle local read successes") { + val transfer = mock(classOf[BlockTransferService]) + val blockManager = mock(classOf[BlockManager]) + doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + + // All blocks should be fetched successfully + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. + verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 1st element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 2nd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 3rd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 4th element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 5th element is not actually defined") + + verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) + } + + test("handle remote fetch failures in BlockTransferService") { + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + listener.onBlockFetchFailure(new Exception("blah")) + } + }) + + val blockManager = mock(classOf[BlockManager]) + + when(blockManager.blockManagerId).thenReturn(BlockManagerId("test-client", "test-client", 1)) + + val blId1 = ShuffleBlockId(0, 0, 0) + val blId2 = ShuffleBlockId(0, 1, 0) + val bmId = BlockManagerId("test-server", "test-server", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L)))) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + iterator.foreach { case (_, iterOption) => + assert(!iterOption.isDefined) + } + } +} From 7db53391f1b349d1f49844197b34f94806f5e336 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 8 Sep 2014 16:14:32 -0700 Subject: [PATCH 08/76] [SPARK-3349][SQL] Output partitioning of limit should not be inherited from child This resolves https://issues.apache.org/jira/browse/SPARK-3349 Author: Eric Liang Closes #2262 from ericl/spark-3349 and squashes the following commits: 3e1b05c [Eric Liang] add regression test ac32723 [Eric Liang] make limit/takeOrdered output SinglePartition --- .../spark/sql/execution/basicOperators.scala | 4 +++- .../org/apache/spark/sql/SQLQuerySuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 47bff0c730b8a..cac376608be29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -27,7 +27,7 @@ import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, SinglePartition, UnspecifiedDistribution} import org.apache.spark.util.MutablePair /** @@ -100,6 +100,7 @@ case class Limit(limit: Int, child: SparkPlan) private def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] override def output = child.output + override def outputPartitioning = SinglePartition /** * A custom implementation modeled after the take function on RDDs but which never runs any job @@ -173,6 +174,7 @@ case class Limit(limit: Int, child: SparkPlan) case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) extends UnaryNode { override def output = child.output + override def outputPartitioning = SinglePartition val ordering = new RowOrdering(sortOrder, child.output) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1ac205937714c..e8fbc28d0ad60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -359,6 +359,23 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (null, null, 6, "F") :: Nil) } + test("SPARK-3349 partitioning after limit") { + sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") + .limit(2) + .registerTempTable("subset1") + sql("SELECT DISTINCT n FROM lowerCaseData") + .limit(2) + .registerTempTable("subset2") + checkAnswer( + sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), + (3, "c", 3) :: + (4, "d", 4) :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), + (1, "a", 1) :: + (2, "b", 2) :: Nil) + } + test("mixed-case keywords") { checkAnswer( sql( From 50a4fa774a0e8a17d7743b33ce8941bf4041144d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 8 Sep 2014 18:59:57 -0700 Subject: [PATCH 09/76] [SPARK-3443][MLLIB] update default values of tree: Adjust the default values of decision tree, based on the memory requirement discussed in https://github.com/apache/spark/pull/2125 : 1. maxMemoryInMB: 128 -> 256 2. maxBins: 100 -> 32 3. maxDepth: 4 -> 5 (in some example code) jkbradley Author: Xiangrui Meng Closes #2322 from mengxr/tree-defaults and squashes the following commits: cda453a [Xiangrui Meng] fix tests 5900445 [Xiangrui Meng] update comments 8c81831 [Xiangrui Meng] update default values of tree: --- docs/mllib-decision-tree.md | 16 ++++++++-------- .../spark/examples/mllib/JavaDecisionTree.java | 2 +- .../examples/mllib/DecisionTreeRunner.scala | 4 ++-- .../apache/spark/mllib/tree/DecisionTree.scala | 8 ++++---- .../mllib/tree/configuration/Strategy.scala | 6 +++--- .../spark/mllib/tree/DecisionTreeSuite.scala | 18 ++++-------------- python/pyspark/mllib/tree.py | 4 ++-- 7 files changed, 24 insertions(+), 34 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 1166d9cd150c4..12a6afbeea829 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -80,7 +80,7 @@ The ordered splits create "bins" and the maximum number of such bins can be specified using the `maxBins` parameter. Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario -since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of +since the default `maxBins` value is 32). The tree algorithm automatically reduces the number of bins if the condition is not satisfied. **Categorical features** @@ -117,7 +117,7 @@ all nodes at each level of the tree. This could lead to high memory requirements of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` training parameter specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to -be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements +be 256 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each subsequent level are split into smaller tasks. @@ -167,7 +167,7 @@ val numClasses = 2 val categoricalFeaturesInfo = Map[Int, Int]() val impurity = "gini" val maxDepth = 5 -val maxBins = 100 +val maxBins = 32 val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, maxDepth, maxBins) @@ -213,7 +213,7 @@ Integer numClasses = 2; HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; -Integer maxBins = 100; +Integer maxBins = 32; // Train a DecisionTree model for classification. final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, @@ -250,7 +250,7 @@ data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, - impurity='gini', maxDepth=5, maxBins=100) + impurity='gini', maxDepth=5, maxBins=32) # Evaluate model on training instances and compute training error predictions = model.predict(data.map(lambda x: x.features)) @@ -293,7 +293,7 @@ val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache val categoricalFeaturesInfo = Map[Int, Int]() val impurity = "variance" val maxDepth = 5 -val maxBins = 100 +val maxBins = 32 val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, maxDepth, maxBins) @@ -338,7 +338,7 @@ JavaSparkContext sc = new JavaSparkContext(sparkConf); HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "variance"; Integer maxDepth = 5; -Integer maxBins = 100; +Integer maxBins = 32; // Train a DecisionTree model. final DecisionTreeModel model = DecisionTree.trainRegressor(data, @@ -380,7 +380,7 @@ data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, - impurity='variance', maxDepth=5, maxBins=100) + impurity='variance', maxDepth=5, maxBins=32) # Evaluate model on training instances and compute training error predictions = model.predict(data.map(lambda x: x.features)) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java index e4468e8bf1744..1f82e3f4cb18e 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java @@ -63,7 +63,7 @@ public static void main(String[] args) { HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; - Integer maxBins = 100; + Integer maxBins = 32; // Train a DecisionTree model for classification. final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index cf3d2cca81ff6..72c3ab475b61f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -52,9 +52,9 @@ object DecisionTreeRunner { input: String = null, dataFormat: String = "libsvm", algo: Algo = Classification, - maxDepth: Int = 4, + maxDepth: Int = 5, impurity: ImpurityType = Gini, - maxBins: Int = 100, + maxBins: Int = 32, fracTest: Double = 0.2) def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index dd766c12d28a4..d1309b2b20f54 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -330,9 +330,9 @@ object DecisionTree extends Serializable with Logging { * Supported values: "gini" (recommended) or "entropy". * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) + * (suggested value: 5) * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) + * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction */ def trainClassifier( @@ -374,9 +374,9 @@ object DecisionTree extends Serializable with Logging { * Supported values: "variance". * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) + * (suggested value: 5) * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) + * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction */ def trainRegressor( 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 cfc8192a85abd..23f74d5360fe5 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 @@ -50,7 +50,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is - * 128 MB. + * 256 MB. */ @Experimental class Strategy ( @@ -58,10 +58,10 @@ class Strategy ( val impurity: Impurity, val maxDepth: Int, val numClassesForClassification: Int = 2, - val maxBins: Int = 100, + val maxBins: Int = 32, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - val maxMemoryInMB: Int = 128) extends Serializable { + val maxMemoryInMB: Int = 256) extends Serializable { if (algo == Classification) { require(numClassesForClassification >= 2) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 8e556c917b2e7..69482f2acbb40 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext - class DecisionTreeSuite extends FunSuite with LocalSparkContext { def validateClassifier( @@ -353,8 +352,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, @@ -381,8 +378,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -410,8 +405,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -439,8 +432,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -464,8 +455,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) @@ -600,7 +589,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3) + numClassesForClassification = 3, maxBins = 100) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -626,7 +615,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) + numClassesForClassification = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(metadata.isUnordered(featureIndex = 0)) @@ -652,7 +641,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + numClassesForClassification = 3, maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index a2fade61e9a71..ccc000ac70ba6 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -138,7 +138,7 @@ class DecisionTree(object): @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=4, maxBins=100): + impurity="gini", maxDepth=5, maxBins=32): """ Train a DecisionTreeModel for classification. @@ -170,7 +170,7 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, @staticmethod def trainRegressor(data, categoricalFeaturesInfo, - impurity="variance", maxDepth=4, maxBins=100): + impurity="variance", maxDepth=5, maxBins=32): """ Train a DecisionTreeModel for regression. From ca0348e68213c2c7589f2018ebf9d889c0ce59c3 Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 8 Sep 2014 19:05:02 -0700 Subject: [PATCH 10/76] SPARK-3423: [SQL] Implement BETWEEN for SQLParser This patch improves the SQLParser by adding support for BETWEEN conditions Author: William Benton Closes #2295 from willb/sql-between and squashes the following commits: 0016d30 [William Benton] Implement BETWEEN for SQLParser --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 ++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 18 ++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a88bd859fc85e..bfc197cf7a938 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -73,6 +73,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val ASC = Keyword("ASC") protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") + protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") protected val CACHE = Keyword("CACHE") protected val CAST = Keyword("CAST") @@ -272,6 +273,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers { termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | + termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { + case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + } | termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e8fbc28d0ad60..45c0ca8ea101d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -597,4 +597,22 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null) :: (4, 2147483644) :: Nil) } + + test("SPARK-3423 BETWEEN") { + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"), + Seq((5, "5"), (6, "6"), (7, "7")) + ) + + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"), + Seq((7, "7")) + ) + + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), + Seq() + ) + + } } From dc1dbf206e0076a43ad2120d8bb5b1fc6912fe25 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 8 Sep 2014 19:08:05 -0700 Subject: [PATCH 11/76] [SPARK-3414][SQL] Stores analyzed logical plan when registering a temp table Case insensitivity breaks when unresolved relation contains attributes with uppercase letters in their names, because we store unanalyzed logical plan when registering temp tables while the `CaseInsensitivityAttributeReferences` batch runs before the `Resolution` batch. To fix this issue, we need to store analyzed logical plan. Author: Cheng Lian Closes #2293 from liancheng/spark-3414 and squashes the following commits: d9fa1d6 [Cheng Lian] Stores analyzed logical plan when registering a temp table --- .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++++++++++++++--- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5acb45c155ba5..a2f334aab9fdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -246,7 +246,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.logicalPlan) + catalog.registerTable(None, tableName, rdd.queryExecution.analyzed) } /** @@ -411,7 +411,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } - def simpleString: String = + def simpleString: String = s"""== Physical Plan == |${stringOrError(executedPlan)} """ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index f4217a52c3822..305998c150327 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,11 +17,8 @@ package org.apache.spark.sql.hive.execution -import java.io.File - import scala.util.Try -import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -514,6 +511,28 @@ class HiveQuerySuite extends HiveComparisonTest { sql("DROP TABLE alter1") } + case class LogEntry(filename: String, message: String) + case class LogFile(name: String) + + test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { + sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") + + sql( + """ + SELECT name, message + FROM rawLogs + JOIN ( + SELECT name + FROM logFiles + ) files + ON rawLogs.filename = files.name + """).registerTempTable("boom") + + // This should be successfully analyzed + sql("SELECT * FROM boom").queryExecution.analyzed + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" From 2b7ab814f9bde65ebc57ebd04386e56c97f06f4a Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 8 Sep 2014 19:29:18 -0700 Subject: [PATCH 12/76] [SPARK-3329][SQL] Don't depend on Hive SET pair ordering in tests. This fixes some possible spurious test failures in `HiveQuerySuite` by comparing sets of key-value pairs as sets, rather than as lists. Author: William Benton Author: Aaron Davidson Closes #2220 from willb/spark-3329 and squashes the following commits: 3b3e205 [William Benton] Collapse collectResults case match in HiveQuerySuite 6525d8e [William Benton] Handle cases where SET returns Rows of (single) strings cf11b0e [Aaron Davidson] Fix flakey HiveQuerySuite test --- .../sql/hive/execution/HiveQuerySuite.scala | 47 ++++++++++--------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 305998c150327..6bf8d18a5c32c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -558,62 +558,67 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - + val KV = "([^=]+)=([^=]*)".r + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { + case Row(key: String, value: String) => key -> value + case Row(KV(key, value)) => key -> value + }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql(s"SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() From 092e2f152fb674e7200cc8a2cb99a8fe0a9b2b33 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Sep 2014 20:51:56 -0700 Subject: [PATCH 13/76] SPARK-2425 Don't kill a still-running Application because of some misbehaving Executors Introduces a LOADING -> RUNNING ApplicationState transition and prevents Master from removing an Application with RUNNING Executors. Two basic changes: 1) Instead of allowing MAX_NUM_RETRY abnormal Executor exits over the entire lifetime of the Application, allow that many since any Executor successfully began running the Application; 2) Don't remove the Application while Master still thinks that there are RUNNING Executors. This should be fine as long as the ApplicationInfo doesn't believe any Executors are forever RUNNING when they are not. I think that any non-RUNNING Executors will eventually no longer be RUNNING in Master's accounting, but another set of eyes should confirm that. This PR also doesn't try to detect which nodes have gone rogue or to kill off bad Workers, so repeatedly failing Executors will continue to fail and fill up log files with failure reports as long as the Application keeps running. Author: Mark Hamstra Closes #1360 from markhamstra/SPARK-2425 and squashes the following commits: f099c0b [Mark Hamstra] Reuse appInfo b2b7b25 [Mark Hamstra] Moved 'Application failed' logging bdd0928 [Mark Hamstra] switched to string interpolation 1dd591b [Mark Hamstra] SPARK-2425 introduce LOADING -> RUNNING ApplicationState transition and prevent Master from removing Application with RUNNING Executors --- .../spark/deploy/master/ApplicationInfo.scala | 4 ++- .../apache/spark/deploy/master/Master.scala | 26 ++++++++++++------- .../spark/deploy/worker/ExecutorRunner.scala | 2 ++ .../apache/spark/deploy/worker/Worker.scala | 2 +- 4 files changed, 22 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index d3674427b1271..c3ca43f8d0734 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -96,11 +96,13 @@ private[spark] class ApplicationInfo( def retryCount = _retryCount - def incrementRetryCount = { + def incrementRetryCount() = { _retryCount += 1 _retryCount } + def resetRetryCount() = _retryCount = 0 + def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2a66fcfe4801c..a3909d6ea95c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -296,28 +296,34 @@ private[spark] class Master( val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId)) execOption match { case Some(exec) => { + val appInfo = idToApp(appId) exec.state = state + if (state == ExecutorState.RUNNING) { appInfo.resetRetryCount() } exec.application.driver ! ExecutorUpdated(execId, state, message, exitStatus) if (ExecutorState.isFinished(state)) { - val appInfo = idToApp(appId) // Remove this executor from the worker and app - logInfo("Removing executor " + exec.fullId + " because it is " + state) + logInfo(s"Removing executor ${exec.fullId} because it is $state") appInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) - val normalExit = exitStatus.exists(_ == 0) + val normalExit = exitStatus == Some(0) // Only retry certain number of times so we don't go into an infinite loop. - if (!normalExit && appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { - schedule() - } else if (!normalExit) { - logError("Application %s with ID %s failed %d times, removing it".format( - appInfo.desc.name, appInfo.id, appInfo.retryCount)) - removeApplication(appInfo, ApplicationState.FAILED) + if (!normalExit) { + if (appInfo.incrementRetryCount() < ApplicationState.MAX_NUM_RETRY) { + schedule() + } else { + val execs = appInfo.executors.values + if (!execs.exists(_.state == ExecutorState.RUNNING)) { + logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + + s"${appInfo.retryCount} times; removing it") + removeApplication(appInfo, ApplicationState.FAILED) + } + } } } } case None => - logWarning("Got status update for unknown executor " + appId + "/" + execId) + logWarning(s"Got status update for unknown executor $appId/$execId") } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 7be89f9aff0f3..00a43673e5cd3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -159,6 +159,8 @@ private[spark] class ExecutorRunner( Files.write(header, stderr, Charsets.UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) + state = ExecutorState.RUNNING + worker ! ExecutorStateChanged(appId, execId, state, None, None) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index e475567db6a20..0c454e4138c96 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -234,7 +234,7 @@ private[spark] class Worker( try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.RUNNING) + self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ From ce5cb325877e3fa8281ffe2076f93b4124ed0eb5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 9 Sep 2014 00:50:59 -0700 Subject: [PATCH 14/76] [Build] Removed -Phive-thriftserver since this profile has been removed Author: Cheng Lian Closes #2269 from liancheng/clean-run-tests-profile and squashes the following commits: 08617bd [Cheng Lian] Removed -Phive-thriftserver since this profile has been removed --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 49a88085c80f7..79401213a7fa2 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -93,7 +93,7 @@ echo "=========================================================================" # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver " +BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive " echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" From c419e4f1bd2a50c558179b7118c3fe75a94b7a5b Mon Sep 17 00:00:00 2001 From: Mario Pastorelli Date: Tue, 9 Sep 2014 00:51:28 -0700 Subject: [PATCH 15/76] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2 Comment of the storageLevel param of actorStream says that it defaults to memory-only while the default is MEMORY_AND_DISK_SER_2. Author: Mario Pastorelli Closes #2319 from melrief/master and squashes the following commits: 7b6ce68 [Mario Pastorelli] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2 --- .../scala/org/apache/spark/streaming/StreamingContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 101cec1c7a7c2..457e8ab28ed82 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -240,7 +240,7 @@ class StreamingContext private[streaming] ( * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param props Props object defining creation of the actor * @param name Name of the actor - * @param storageLevel RDD storage level. Defaults to memory-only. + * @param storageLevel RDD storage level (default: StorageLevel.MEMORY_AND_DISK_SER_2) * * @note An important point to note: * Since Actor may exist outside the spark framework, It is thus user's responsibility From 1e03cf79f82b166b2e18dcbd181e074f0276a0a9 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 9 Sep 2014 10:18:25 -0700 Subject: [PATCH 16/76] [SPARK-3455] [SQL] **HOT FIX** Fix the unit test failure Unit test failed due to can not resolve the attribute references. Temporally disable this test case for a quick fixing, otherwise it will block the others. Author: Cheng Hao Closes #2334 from chenghao-intel/unit_test_failure and squashes the following commits: 661f784 [Cheng Hao] temporally disable the failed test case --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 45c0ca8ea101d..739c12f338f34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -360,6 +360,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3349 partitioning after limit") { + /* sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") @@ -374,6 +375,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), (1, "a", 1) :: (2, "b", 2) :: Nil) + */ } test("mixed-case keywords") { From 88547a09fcc25df132b401ecec4ebe1ef6778576 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Sep 2014 10:23:28 -0700 Subject: [PATCH 17/76] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere. Author: Sandy Ryza Closes #2324 from sryza/sandy-spark-3422 and squashes the following commits: 6446175 [Sandy Ryza] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere. --- .../java/org/apache/spark/JavaAPISuite.java | 25 ------------------- 1 file changed, 25 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index be99dc501c4b2..b8574dfb42e6b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -29,19 +29,14 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; -import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapreduce.Job; import org.junit.After; import org.junit.Assert; @@ -49,7 +44,6 @@ import org.junit.Test; import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaHadoopRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -1313,23 +1307,4 @@ public void collectUnderlyingScalaRDD() { SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); Assert.assertEquals(data.size(), collected.length); } - - public void getHadoopInputSplits() { - String outDir = new File(tempDir, "output").getAbsolutePath(); - sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2).saveAsTextFile(outDir); - - JavaHadoopRDD hadoopRDD = (JavaHadoopRDD) - sc.hadoopFile(outDir, TextInputFormat.class, LongWritable.class, Text.class); - List inputPaths = hadoopRDD.mapPartitionsWithInputSplit( - new Function2>, Iterator>() { - @Override - public Iterator call(InputSplit split, Iterator> it) - throws Exception { - FileSplit fileSplit = (FileSplit) split; - return Lists.newArrayList(fileSplit.getPath().toUri().getPath()).iterator(); - } - }, true).collect(); - Assert.assertEquals(Sets.newHashSet(inputPaths), - Sets.newHashSet(outDir + "/part-00000", outDir + "/part-00001")); - } } From f0f1ba09b195f23f0c89af6fa040c9e01dfa8951 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 9 Sep 2014 10:24:00 -0700 Subject: [PATCH 18/76] SPARK-3404 [BUILD] SparkSubmitSuite fails with "spark-submit exits with code 1" This fixes the `SparkSubmitSuite` failure by setting `0` in the Maven build, to match the SBT build. This avoids a port conflict which causes failures. (This also updates the `scalatest` plugin off of a release candidate, to the identical final release.) Author: Sean Owen Closes #2328 from srowen/SPARK-3404 and squashes the following commits: 512d782 [Sean Owen] Set spark.ui.port=0 in Maven scalatest config to match SBT build and avoid SparkSubmitSuite failure due to port conflict --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d05190512f742..64fb1e57e30e0 100644 --- a/pom.xml +++ b/pom.xml @@ -888,7 +888,7 @@ org.scalatest scalatest-maven-plugin - 1.0-RC2 + 1.0 ${project.build.directory}/surefire-reports . @@ -899,6 +899,7 @@ true ${session.executionRootDirectory} 1 + 0 From 26862337c97ce14794178d6378fb4155dd24acb9 Mon Sep 17 00:00:00 2001 From: scwf Date: Tue, 9 Sep 2014 11:57:01 -0700 Subject: [PATCH 19/76] [SPARK-3193]output errer info when Process exit code is not zero in test suite https://issues.apache.org/jira/browse/SPARK-3193 I noticed that sometimes pr tests failed due to the Process exitcode != 0,refer to https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18688/consoleFull https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19118/consoleFull [info] SparkSubmitSuite: [info] - prints usage on empty input [info] - prints usage with only --help [info] - prints error with unrecognized options [info] - handle binary specified but not class [info] - handles arguments with --key=val [info] - handles arguments to user program [info] - handles arguments to user program with name collision [info] - handles YARN cluster mode [info] - handles YARN client mode [info] - handles standalone cluster mode [info] - handles standalone client mode [info] - handles mesos client mode [info] - handles confs with flag equivalents [info] - launch simple application with spark-submit *** FAILED *** [info] org.apache.spark.SparkException: Process List(./bin/spark-submit, --class, org.apache.spark.deploy.SimpleApplicationTest, --name, testApp, --master, local, file:/tmp/1408854098404-0/testJar-1408854098404.jar) exited with code 1 [info] at org.apache.spark.util.Utils$.executeAndGetOutput(Utils.scala:872) [info] at org.apache.spark.deploy.SparkSubmitSuite.runSparkSubmit(SparkSubmitSuite.scala:311) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply$mcV$sp(SparkSubmitSuite.scala:291) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply(SparkSubmitSuite.scala:284) [info] at org.apacSpark assembly has been built with Hive, including Datanucleus jars on classpath this PR output the process error info when failed, it can be helpful for diagnosis. Author: scwf Closes #2108 from scwf/output-test-error-info and squashes the following commits: 0c48082 [scwf] minor fix according to comments 563fde1 [scwf] output errer info when Process exitcode not zero --- .../scala/org/apache/spark/util/Utils.scala | 19 ++++++++++++++++++- .../scala/org/apache/spark/DriverSuite.scala | 5 +---- .../spark/deploy/SparkSubmitSuite.scala | 2 ++ 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0ae28f911e302..79943766d0f0f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -20,9 +20,11 @@ package org.apache.spark.util import java.io._ import java.net._ import java.nio.ByteBuffer -import java.util.{Locale, Random, UUID} +import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import org.apache.log4j.PropertyConfigurator + import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -834,6 +836,7 @@ private[spark] object Utils extends Logging { val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output if (exitCode != 0) { + logError(s"Process $command exited with code $exitCode: ${output}") throw new SparkException("Process " + command + " exited with code " + exitCode) } output.toString @@ -1444,6 +1447,20 @@ private[spark] object Utils extends Logging { } } + /** + * config a log4j properties used for testsuite + */ + def configTestLog4j(level: String): Unit = { + val pro = new Properties() + pro.put("log4j.rootLogger", s"$level, console") + pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender") + pro.put("log4j.appender.console.target", "System.err") + pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout") + pro.put("log4j.appender.console.layout.ConversionPattern", + "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n") + PropertyConfigurator.configure(pro) + } + } /** diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index a73e1ef0288a5..4b1d280624c57 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark import java.io.File -import org.apache.log4j.Logger -import org.apache.log4j.Level - import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -54,7 +51,7 @@ class DriverSuite extends FunSuite with Timeouts { */ object DriverWithoutCleanup { def main(args: Array[String]) { - Logger.getRootLogger().setLevel(Level.WARN) + Utils.configTestLog4j("INFO") val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 7e1ef80c84561..22b369a829418 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -317,6 +317,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { object JarCreationTest { def main(args: Array[String]) { + Utils.configTestLog4j("INFO") val conf = new SparkConf() val sc = new SparkContext(conf) val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => @@ -338,6 +339,7 @@ object JarCreationTest { object SimpleApplicationTest { def main(args: Array[String]) { + Utils.configTestLog4j("INFO") val conf = new SparkConf() val sc = new SparkContext(conf) val configs = Seq("spark.master", "spark.app.name") From 02b5ac7191c66a866ffedde313eb10f2adfc9b58 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 9 Sep 2014 14:42:28 -0700 Subject: [PATCH 20/76] Minor - Fix trivial compilation warnings. Author: Prashant Sharma Closes #2331 from ScrapCodes/compilation-warn and squashes the following commits: 44c1e76 [Prashant Sharma] Minor - Fix trivial compilation warnings. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 - .../org/apache/spark/examples/graphx/LiveJournalPageRank.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 24d1a8f9eceae..c6c5b8f22b549 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,7 +49,6 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ -import org.apache.spark.SPARK_VERSION import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index bdc8fa7f99f2e..e809a65b79975 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ import org.apache.spark._ import org.apache.spark.graphx._ -import org.apache.spark.examples.graphx.Analytics + /** * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from From 07ee4a28c3a502121770f301316cb2256e8f0ce2 Mon Sep 17 00:00:00 2001 From: xinyunh Date: Tue, 9 Sep 2014 16:55:39 -0700 Subject: [PATCH 21/76] [SPARK-3176] Implement 'ABS and 'LAST' for sql Add support for the mathematical function"ABS" and the analytic function "last" to return a subset of the rows satisfying a query within spark sql. Test-cases included. Author: xinyunh Author: bomeng Closes #2099 from xinyunh/sqlTest and squashes the following commits: 71d15e7 [xinyunh] remove POWER part 8843643 [xinyunh] fix the code style issue 39f0309 [bomeng] Modify the code of POWER and ABS. Move them to the file arithmetic ff8e51e [bomeng] add abs() function support 7f6980a [xinyunh] fix the bug in 'Last' component b3df91b [xinyunh] add 'Last' component --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +++ .../spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/expressions/aggregates.scala | 28 +++++++++++++++++++ .../sql/catalyst/expressions/arithmetic.scala | 15 ++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 23 +++++++++++++-- 5 files changed, 69 insertions(+), 2 deletions(-) mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala old mode 100644 new mode 100755 index bfc197cf7a938..a04b4a938da64 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -82,6 +82,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val DISTINCT = Keyword("DISTINCT") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") + protected val LAST = Keyword("LAST") protected val FROM = Keyword("FROM") protected val FULL = Keyword("FULL") protected val GROUP = Keyword("GROUP") @@ -125,6 +126,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val SUBSTR = Keyword("SUBSTR") protected val SUBSTRING = Keyword("SUBSTRING") protected val SQRT = Keyword("SQRT") + protected val ABS = Keyword("ABS") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -315,6 +317,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) } | FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | + LAST ~> "(" ~> expression <~ ")" ^^ { case exp => Last(exp) } | AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | MAX ~> "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } | @@ -330,6 +333,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { case s ~ "," ~ p ~ "," ~ l => Substring(s,p,l) } | SQRT ~> "(" ~> expression <~ ")" ^^ { case exp => Sqrt(exp) } | + ABS ~> "(" ~> expression <~ ")" ^^ { case exp => Abs(exp) } | ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala old mode 100644 new mode 100755 index f44521d6381c9..deb622c39faf5 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -132,6 +132,7 @@ package object dsl { def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) def avg(e: Expression) = Average(e) def first(e: Expression) = First(e) + def last(e: Expression) = Last(e) def min(e: Expression) = Min(e) def max(e: Expression) = Max(e) def upper(e: Expression) = Upper(e) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala old mode 100644 new mode 100755 index 15560a2a933ad..1b4d892625dbb --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -344,6 +344,21 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod override def newInstance() = new FirstFunction(child, this) } +case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = true + override def dataType = child.dataType + override def toString = s"LAST($child)" + + override def asPartial: SplitEvaluation = { + val partialLast = Alias(Last(child), "PartialLast")() + SplitEvaluation( + Last(partialLast.toAttribute), + partialLast :: Nil) + } + override def newInstance() = new LastFunction(child, this) +} + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -489,3 +504,16 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag override def eval(input: Row): Any = result } + +case class LastFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var result: Any = null + + override def update(input: Row): Unit = { + result = input + } + + override def eval(input: Row): Any = if (result != null) expr.eval(result.asInstanceOf[Row]) + else null +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index f988fb010b107..fe825fdcdae37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ +import scala.math.pow case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any @@ -129,3 +130,17 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def toString = s"MaxOf($left, $right)" } + +/** + * A function that get the absolute value of the numeric value. + */ +case class Abs(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def dataType = child.dataType + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"Abs($child)" + + override def eval(input: Row): Any = n1(child, input, _.abs(_)) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 739c12f338f34..514ac543df92a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -41,6 +41,25 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } + test("SPARK-3176 Added Parser of SQL ABS()") { + checkAnswer( + sql("SELECT ABS(-1.3)"), + 1.3) + checkAnswer( + sql("SELECT ABS(0.0)"), + 0.0) + checkAnswer( + sql("SELECT ABS(2.5)"), + 2.5) + } + + test("SPARK-3176 Added Parser of SQL LAST()") { + checkAnswer( + sql("SELECT LAST(n) FROM lowerCaseData"), + 4) + } + + test("SPARK-2041 column name equals tablename") { checkAnswer( sql("SELECT tableName FROM tableName"), @@ -53,14 +72,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq ) } - + test("SQRT with automatic string casts") { checkAnswer( sql("SELECT SQRT(CAST(key AS STRING)) FROM testData"), (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq ) } - + test("SPARK-2407 Added Parser of SQL SUBSTR()") { checkAnswer( sql("SELECT substr(tableName, 1, 2) FROM tableName"), From c110614b33a690a3db6ccb1a920fb6a3795aa5a0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 9 Sep 2014 18:39:33 -0700 Subject: [PATCH 22/76] [SPARK-3448][SQL] Check for null in SpecificMutableRow.update `SpecificMutableRow.update` doesn't check for null, and breaks existing `MutableRow` contract. The tricky part here is that for performance considerations, the `update` method of all subclasses of `MutableValue` doesn't check for null and sets the null bit to false. Author: Cheng Lian Closes #2325 from liancheng/check-for-null and squashes the following commits: 9366c44 [Cheng Lian] Check for null in SpecificMutableRow.update --- .../spark/sql/catalyst/expressions/SpecificRow.scala | 4 +++- .../src/test/scala/org/apache/spark/sql/RowSuite.scala | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala index 75ea0e8459df8..088f11ee4aa53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -227,7 +227,9 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new SpecificMutableRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = values(ordinal).update(value) + override def update(ordinal: Int, value: Any): Unit = { + if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + } override def iterator: Iterator[Any] = values.map(_.boxed).iterator diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 651cb735ab7d9..811319e0a6601 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} class RowSuite extends FunSuite { @@ -43,4 +43,10 @@ class RowSuite extends FunSuite { assert(expected.getBoolean(2) === actual2.getBoolean(2)) assert(expected(3) === actual2(3)) } + + test("SpecificMutableRow.update with null") { + val row = new SpecificMutableRow(Seq(IntegerType)) + row(0) = null + assert(row.isNullAt(0)) + } } From 25b5b867d5e18bac1c5bcdc6f8c63d97858194c7 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Tue, 9 Sep 2014 18:54:54 -0700 Subject: [PATCH 23/76] [SPARK-3458] enable python "with" statements for SparkContext allow for best practice code, ``` try: sc = SparkContext() app(sc) finally: sc.stop() ``` to be written using a "with" statement, ``` with SparkContext() as sc: app(sc) ``` Author: Matthew Farrellee Closes #2335 from mattf/SPARK-3458 and squashes the following commits: 5b4e37c [Matthew Farrellee] [SPARK-3458] enable python "with" statements for SparkContext --- python/pyspark/context.py | 14 ++++++++++++++ python/pyspark/tests.py | 29 +++++++++++++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5a30431568b16..84bc0a3b7ccd0 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -232,6 +232,20 @@ def _ensure_initialized(cls, instance=None, gateway=None): else: SparkContext._active_spark_context = instance + def __enter__(self): + """ + Enable 'with SparkContext(...) as sc: app(sc)' syntax. + """ + return self + + def __exit__(self, type, value, trace): + """ + Enable 'with SparkContext(...) as sc: app' syntax. + + Specifically stop the context on exit of the with block. + """ + self.stop() + @classmethod def setSystemProperty(cls, key, value): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0bd2a9e6c507d..bb84ebe72cb24 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1254,6 +1254,35 @@ def test_single_script_on_cluster(self): self.assertIn("[2, 4, 6]", out) +class ContextStopTests(unittest.TestCase): + + def test_stop(self): + sc = SparkContext() + self.assertNotEqual(SparkContext._active_spark_context, None) + sc.stop() + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with(self): + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with_exception(self): + try: + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + raise Exception() + except: + pass + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with_stop(self): + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + sc.stop() + self.assertEqual(SparkContext._active_spark_context, None) + + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): From b734ed0c229373dbc589b9eca7327537ca458138 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 9 Sep 2014 23:47:12 -0700 Subject: [PATCH 24/76] [SPARK-3395] [SQL] DSL sometimes incorrectly reuses attribute ids, breaking queries This resolves https://issues.apache.org/jira/browse/SPARK-3395 Author: Eric Liang Closes #2266 from ericl/spark-3395 and squashes the following commits: 7f2b6f0 [Eric Liang] add regression test 05bd1e4 [Eric Liang] in the dsl, create a new schema instance in each applySchema --- .../scala/org/apache/spark/sql/SchemaRDD.scala | 3 ++- .../scala/org/apache/spark/sql/DslQuerySuite.scala | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 33b2ed1b3a399..d2ceb4a2b0b25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -428,7 +428,8 @@ class SchemaRDD( */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { new SchemaRDD(sqlContext, - SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))(sqlContext)) + SparkLogicalPlan( + ExistingRdd(queryExecution.analyzed.output.map(_.newInstance), rdd))(sqlContext)) } // ======================================================================= diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 1a6a6c17473a3..d001abb7e1fcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.test._ /* Implicits */ @@ -133,6 +135,18 @@ class DslQuerySuite extends QueryTest { mapData.take(1).toSeq) } + test("SPARK-3395 limit distinct") { + val filtered = TestData.testData2 + .distinct() + .orderBy(SortOrder('a, Ascending), SortOrder('b, Ascending)) + .limit(1) + .registerTempTable("onerow") + checkAnswer( + sql("select * from onerow inner join testData2 on onerow.a = testData2.a"), + (1, 1, 1, 1) :: + (1, 1, 1, 2) :: Nil) + } + test("average") { checkAnswer( testData2.groupBy()(avg('a)), From 6f7a76838f15687583e3b0ab43309a3c079368c4 Mon Sep 17 00:00:00 2001 From: Benoy Antony Date: Wed, 10 Sep 2014 11:59:39 -0500 Subject: [PATCH 25/76] =?UTF-8?q?[SPARK-3286]=20-=20Cannot=20view=20Applic?= =?UTF-8?q?ationMaster=20UI=20when=20Yarn=E2=80=99s=20url=20scheme=20i...?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ...s https Author: Benoy Antony Closes #2276 from benoyantony/SPARK-3286 and squashes the following commits: c3d51ee [Benoy Antony] Use address with scheme, but Allpha version removes the scheme e82f94e [Benoy Antony] Use address with scheme, but Allpha version removes the scheme 92127c9 [Benoy Antony] rebasing from master 450c536 [Benoy Antony] [SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url scheme is https f060c02 [Benoy Antony] [SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url scheme is https --- .../scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 4 +++- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index ad27a9ab781d2..fc30953011812 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import scala.collection.{Map, Set} +import java.net.URI; import org.apache.hadoop.net.NetUtils import org.apache.hadoop.yarn.api._ @@ -97,7 +98,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) - appMasterRequest.setTrackingUrl(uiAddress) + //remove the scheme from the url if it exists since Hadoop does not expect scheme + appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) resourceManager.registerApplicationMaster(appMasterRequest) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a879c833a014f..5756263e89e21 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -189,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort, securityMgr) + registerAM(sc.ui.appUIAddress, securityMgr) try { userThread.join() } finally { From a0283300c4af5e64a1dc06193245daa1e746b5f4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Sep 2014 10:45:15 -0700 Subject: [PATCH 26/76] [SPARK-3362][SQL] Fix resolution for casewhen with nulls. Current implementation will ignore else val type. Author: Daoyuan Wang Closes #2245 from adrian-wang/casewhenbug and squashes the following commits: 3332f6e [Daoyuan Wang] remove wrong comment 83b536c [Daoyuan Wang] a comment to trigger retest d7315b3 [Daoyuan Wang] code improve eed35fc [Daoyuan Wang] bug in casewhen resolve --- .../apache/spark/sql/catalyst/expressions/predicates.scala | 5 +++-- ...then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c | 1 + ...en 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 | 1 + ...hen 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb | 1 + ...hen 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 | 1 + ...hen 1Y else null end -0-589982a400d86157791c7216b10b6b5d | 1 + ...then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 | 1 + ...en null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 | 1 + ...hen null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 | 1 + ...hen null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 | 1 + ...hen null else 1Y end -0-7f4c32299c3738739b678ece62752a7b | 1 + .../spark/sql/hive/execution/HiveTypeCoercionSuite.scala | 6 ++++++ 12 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c create mode 100644 sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 create mode 100644 sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb create mode 100644 sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 create mode 100644 sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 1313ccd120c1f..329af332d0fa1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -265,12 +265,13 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { false } else { val allCondBooleans = predicates.forall(_.dataType == BooleanType) - val dataTypesEqual = values.map(_.dataType).distinct.size <= 1 + // both then and else val should be considered. + val dataTypesEqual = (values ++ elseValue).map(_.dataType).distinct.size <= 1 allCondBooleans && dataTypesEqual } } - /** Written in imperative fashion for performance considerations. Same for CaseKeyWhen. */ + /** Written in imperative fashion for performance considerations. */ override def eval(input: Row): Any = { val len = branchesArr.length var i = 0 diff --git a/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c b/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb b/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 b/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d b/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 b/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 b/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 b/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b b/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index c3c18cf8ccac3..48fffe53cf2ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -33,6 +33,12 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { } } + val nullVal = "null" + baseTypes.init.foreach { i => + createQueryTest(s"case when then $i else $nullVal end ", s"SELECT case when true then $i else $nullVal end FROM src limit 1") + createQueryTest(s"case when then $nullVal else $i end ", s"SELECT case when true then $nullVal else $i end FROM src limit 1") + } + test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" val project = TestHive.sql(q).queryExecution.executedPlan.collect { case e: Project => e }.head From f0c87dc86ae65a39cd19370d8d960b4a60854517 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Sep 2014 10:48:33 -0700 Subject: [PATCH 27/76] [SPARK-3363][SQL] Type Coercion should promote null to all other types. Type Coercion should support every type to have null value Author: Daoyuan Wang Author: Michael Armbrust Closes #2246 from adrian-wang/spark3363-0 and squashes the following commits: c6241de [Daoyuan Wang] minor code clean 595b417 [Daoyuan Wang] Merge pull request #2 from marmbrus/pr/2246 832e640 [Michael Armbrust] reduce code duplication ef6f986 [Daoyuan Wang] make double boolean miss in jsonRDD compatibleType c619f0a [Daoyuan Wang] Type Coercion should support every type to have null value --- .../catalyst/analysis/HiveTypeCoercion.scala | 38 +++++++------- .../analysis/HiveTypeCoercionSuite.scala | 32 +++++++++--- .../org/apache/spark/sql/json/JsonRDD.scala | 51 ++++++++----------- 3 files changed, 67 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index d6758eb5b6a32..bd8131c9af6e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -26,10 +26,22 @@ object HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: val numericPrecedence = - Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) - // Boolean is only wider than Void - val booleanPrecedence = Seq(NullType, BooleanType) - val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) + val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: Nil + + def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { + val valueTypes = Seq(t1, t2).filter(t => t != NullType) + if (valueTypes.distinct.size > 1) { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } else { + Some(if (valueTypes.size == 0) NullType else valueTypes.head) + } + } } /** @@ -53,17 +65,6 @@ trait HiveTypeCoercion { Division :: Nil - trait TypeWidening { - def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - } - } - /** * Applies any changes to [[AttributeReference]] data types that are made by other rules to * instances higher in the query tree. @@ -144,7 +145,8 @@ trait HiveTypeCoercion { * - LongType to FloatType * - LongType to DoubleType */ - object WidenTypes extends Rule[LogicalPlan] with TypeWidening { + object WidenTypes extends Rule[LogicalPlan] { + import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transform { case u @ Union(left, right) if u.childrenResolved && !u.resolved => @@ -352,7 +354,9 @@ trait HiveTypeCoercion { /** * Coerces the type of different branches of a CASE WHEN statement to a common type. */ - object CaseWhenCoercion extends Rule[LogicalPlan] with TypeWidening { + object CaseWhenCoercion extends Rule[LogicalPlan] { + import HiveTypeCoercion._ + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => val valueTypes = branches.sliding(2, 2).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index b9e0f8e9dcc5f..ba8b853b6f99e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -23,20 +23,20 @@ import org.apache.spark.sql.catalyst.types._ class HiveTypeCoercionSuite extends FunSuite { - val rules = new HiveTypeCoercion { } - import rules._ - - test("tightest common bound for numeric and boolean types") { + test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { - var found = WidenTypes.findTightestCommonType(t1, t2) + var found = HiveTypeCoercion.findTightestCommonType(t1, t2) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found") // Test both directions to make sure the widening is symmetric. - found = WidenTypes.findTightestCommonType(t2, t1) + found = HiveTypeCoercion.findTightestCommonType(t2, t1) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") } + // Null + widenTest(NullType, NullType, Some(NullType)) + // Boolean widenTest(NullType, BooleanType, Some(BooleanType)) widenTest(BooleanType, BooleanType, Some(BooleanType)) @@ -60,12 +60,28 @@ class HiveTypeCoercionSuite extends FunSuite { widenTest(DoubleType, DoubleType, Some(DoubleType)) // Integral mixed with floating point. - widenTest(NullType, FloatType, Some(FloatType)) - widenTest(NullType, DoubleType, Some(DoubleType)) widenTest(IntegerType, FloatType, Some(FloatType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) widenTest(LongType, FloatType, Some(FloatType)) widenTest(LongType, DoubleType, Some(DoubleType)) + + // StringType + widenTest(NullType, StringType, Some(StringType)) + widenTest(StringType, StringType, Some(StringType)) + widenTest(IntegerType, StringType, None) + widenTest(LongType, StringType, None) + + // TimestampType + widenTest(NullType, TimestampType, Some(TimestampType)) + widenTest(TimestampType, TimestampType, Some(TimestampType)) + widenTest(IntegerType, TimestampType, None) + widenTest(StringType, TimestampType, None) + + // ComplexType + widenTest(NullType, MapType(IntegerType, StringType, false), Some(MapType(IntegerType, StringType, false))) + widenTest(NullType, StructType(Seq()), Some(StructType(Seq()))) + widenTest(StringType, MapType(IntegerType, StringType, true), None) + widenTest(ArrayType(IntegerType), StructType(Seq()), None) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 1c0b03c684f10..70062eae3b7ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -125,38 +125,31 @@ private[sql] object JsonRDD extends Logging { * Returns the most general data type for two given data types. */ private[json] def compatibleType(t1: DataType, t2: DataType): DataType = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p - .contains(t2)) - - // If found return the widest common type, otherwise None - val returnType = applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - - if (returnType.isDefined) { - returnType.get - } else { - // t1 or t2 is a StructType, ArrayType, or an unexpected type. - (t1, t2) match { - case (other: DataType, NullType) => other - case (NullType, other: DataType) => other - case (StructType(fields1), StructType(fields2)) => { - val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { - case (name, fieldTypes) => { - val dataType = fieldTypes.map(field => field.dataType).reduce( - (type1: DataType, type2: DataType) => compatibleType(type1, type2)) - StructField(name, dataType, true) + HiveTypeCoercion.findTightestCommonType(t1, t2) match { + case Some(commonType) => commonType + case None => + // t1 or t2 is a StructType, ArrayType, or an unexpected type. + (t1, t2) match { + case (other: DataType, NullType) => other + case (NullType, other: DataType) => other + case (StructType(fields1), StructType(fields2)) => { + val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { + case (name, fieldTypes) => { + val dataType = fieldTypes.map(field => field.dataType).reduce( + (type1: DataType, type2: DataType) => compatibleType(type1, type2)) + StructField(name, dataType, true) + } } + StructType(newFields.toSeq.sortBy { + case StructField(name, _, _) => name + }) } - StructType(newFields.toSeq.sortBy { - case StructField(name, _, _) => name - }) + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) + // TODO: We should use JsonObjectStringType to mark that values of field will be + // strings and every string is a Json object. + case (_, _) => StringType } - case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => - ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) - // TODO: We should use JsonObjectStringType to mark that values of field will be - // strings and every string is a Json object. - case (_, _) => StringType - } } } From 26503fdf20f4181a2b390c88b83f364e6a4ccc21 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 Sep 2014 12:02:23 -0700 Subject: [PATCH 28/76] [HOTFIX] Fix scala style issue introduced by #2276. --- .../scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index fc30953011812..acf26505e4cf9 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -98,7 +98,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) - //remove the scheme from the url if it exists since Hadoop does not expect scheme + // remove the scheme from the url if it exists since Hadoop does not expect scheme appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) resourceManager.registerApplicationMaster(appMasterRequest) } From 1f4a648d4e30e837d6cf3ea8de1808e2254ad70b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 10 Sep 2014 14:34:24 -0500 Subject: [PATCH 29/76] SPARK-1713. Use a thread pool for launching executors. This patch copies the approach used in the MapReduce application master for launching containers. Author: Sandy Ryza Closes #663 from sryza/sandy-spark-1713 and squashes the following commits: 036550d [Sandy Ryza] SPARK-1713. [YARN] Use a threadpool for launching executor containers --- docs/running-on-yarn.md | 7 +++++++ .../apache/spark/deploy/yarn/YarnAllocator.scala | 14 ++++++++++++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 943f06b114cb9..d8b22f3663d08 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -125,6 +125,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes the environment of the executor launcher. + + spark.yarn.containerLauncherMaxThreads + 25 + + The maximum number of threads to use in the application master for launching executor containers. + + # Launching Spark on YARN diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 02b9a81bf6b50..0b8744f4b8bdf 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ @@ -32,6 +32,8 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import com.google.common.util.concurrent.ThreadFactoryBuilder + object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value @@ -95,6 +97,14 @@ private[yarn] abstract class YarnAllocator( protected val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, preferredNodes) + private val launcherPool = new ThreadPoolExecutor( + // max pool size of Integer.MAX_VALUE is ignored because we use an unbounded queue + sparkConf.getInt("spark.yarn.containerLauncherMaxThreads", 25), Integer.MAX_VALUE, + 1, TimeUnit.MINUTES, + new LinkedBlockingQueue[Runnable](), + new ThreadFactoryBuilder().setNameFormat("ContainerLauncher #%d").setDaemon(true).build()) + launcherPool.allowCoreThreadTimeOut(true) + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue def getNumExecutorsFailed: Int = numExecutorsFailed.intValue @@ -283,7 +293,7 @@ private[yarn] abstract class YarnAllocator( executorMemory, executorCores, securityMgr) - new Thread(executorRunnable).start() + launcherPool.execute(executorRunnable) } } logDebug(""" From e4f4886d7148bf48f9e3462b83bfb1ecc7edbe31 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 Sep 2014 12:56:59 -0700 Subject: [PATCH 30/76] [SPARK-2096][SQL] Correctly parse dot notations First let me write down the current `projections` grammar of spark sql: expression : orExpression orExpression : andExpression {"or" andExpression} andExpression : comparisonExpression {"and" comparisonExpression} comparisonExpression : termExpression | termExpression "=" termExpression | termExpression ">" termExpression | ... termExpression : productExpression {"+"|"-" productExpression} productExpression : baseExpression {"*"|"/"|"%" baseExpression} baseExpression : expression "[" expression "]" | ... | ident | ... ident : identChar {identChar | digit} | delimiters | ... identChar : letter | "_" | "." delimiters : "," | ";" | "(" | ")" | "[" | "]" | ... projection : expression [["AS"] ident] projections : projection { "," projection} For something like `a.b.c[1]`, it will be parsed as: But for something like `a[1].b`, the current grammar can't parse it correctly. A simple solution is written in `ParquetQuerySuite#NestedSqlParser`, changed grammars are: delimiters : "." | "," | ";" | "(" | ")" | "[" | "]" | ... identChar : letter | "_" baseExpression : expression "[" expression "]" | expression "." ident | ... | ident | ... This works well, but can't cover some corner case like `select t.a.b from table as t`: `t.a.b` parsed as `GetField(GetField(UnResolved("t"), "a"), "b")` instead of `GetField(UnResolved("t.a"), "b")` using this new grammar. However, we can't resolve `t` as it's not a filed, but the whole table.(if we could do this, then `select t from table as t` is legal, which is unexpected) My solution is: dotExpressionHeader : ident "." ident baseExpression : expression "[" expression "]" | expression "." ident | ... | dotExpressionHeader | ident | ... I passed all test cases under sql locally and add a more complex case. "arrayOfStruct.field1 to access all values of field1" is not supported yet. Since this PR has changed a lot of code, I will open another PR for it. I'm not familiar with the latter optimize phase, please correct me if I missed something. Author: Wenchen Fan Author: Michael Armbrust Closes #2230 from cloud-fan/dot and squashes the following commits: e1a8898 [Wenchen Fan] remove support for arbitrary nested arrays ee8a724 [Wenchen Fan] rollback LogicalPlan, support dot operation on nested array type a58df40 [Michael Armbrust] add regression test for doubly nested data 16bc4c6 [Wenchen Fan] some enhance 95d733f [Wenchen Fan] split long line dc31698 [Wenchen Fan] SPARK-2096 Correctly parse dot notations --- .../apache/spark/sql/catalyst/SqlParser.scala | 13 ++- .../catalyst/plans/logical/LogicalPlan.scala | 6 +- .../org/apache/spark/sql/json/JsonSuite.scala | 14 +++ .../apache/spark/sql/json/TestJsonData.scala | 26 +++++ .../spark/sql/parquet/ParquetQuerySuite.scala | 102 +++++------------- .../sql/hive/execution/SQLQuerySuite.scala | 17 ++- 6 files changed, 88 insertions(+), 90 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a04b4a938da64..ca69531c69a77 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -357,16 +357,25 @@ class SqlParser extends StandardTokenParsers with PackratParsers { expression ~ "[" ~ expression <~ "]" ^^ { case base ~ _ ~ ordinal => GetItem(base, ordinal) } | + (expression <~ ".") ~ ident ^^ { + case base ~ fieldName => GetField(base, fieldName) + } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | function | "-" ~> literal ^^ UnaryMinus | + dotExpressionHeader | ident ^^ UnresolvedAttribute | "*" ^^^ Star(None) | literal + protected lazy val dotExpressionHeader: Parser[Expression] = + (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { + case i1 ~ i2 ~ rest => UnresolvedAttribute(i1 + "." + i2 + rest.mkString(".", ".", "")) + } + protected lazy val dataType: Parser[DataType] = STRING ^^^ StringType | TIMESTAMP ^^^ TimestampType } @@ -380,7 +389,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]" + ",", ";", "%", "{", "}", ":", "[", "]", "." ) override lazy val token: Parser[Token] = ( @@ -401,7 +410,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { | failure("illegal character") ) - override def identChar = letter | elem('_') | elem('.') + override def identChar = letter | elem('_') override def whitespace: Parser[Any] = rep( whitespaceChar diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index f81d9111945f5..bae491f07c13f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -104,11 +104,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - a.dataType match { - case StructType(fields) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case _ => None // Don't know how to resolve these field references - } + Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) case Seq() => None // No matches. case ambiguousReferences => throw new TreeNodeException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 05513a127150c..301d482d27d86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -581,4 +581,18 @@ class JsonSuite extends QueryTest { "this is a simple string.") :: Nil ) } + + test("SPARK-2096 Correctly parse dot notations") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType2) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), + (true, "str1") :: Nil + ) + checkAnswer( + sql("select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] from jsonTable"), + ("str2", 6) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index a88310b5f1b46..b3f95f08e8044 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -82,4 +82,30 @@ object TestJsonData { """{"c":[33, 44]}""" :: """{"d":{"field":true}}""" :: """{"e":"str"}""" :: Nil) + + val complexFieldAndType2 = + TestSQLContext.sparkContext.parallelize( + """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], + "complexArrayOfStruct": [ + { + "field1": [ + { + "inner1": "str1" + }, + { + "inner2": ["str2", "str22"] + }], + "field2": [[1, 2], [3, 4]] + }, + { + "field1": [ + { + "inner2": ["str3", "str33"] + }, + { + "inner1": "str4" + }], + "field2": [[5, 6], [7, 8]] + }] + }""" :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 42923b6a288d9..b0a06cd3ca090 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.parquet +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} - import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.mapreduce.Job - -import org.apache.spark.SparkContext import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType} +import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ @@ -87,11 +82,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA var testRDD: SchemaRDD = null - // TODO: remove this once SqlParser can parse nested select statements - var nestedParserSqlContext: NestedParserSQLContext = null - override def beforeAll() { - nestedParserSqlContext = new NestedParserSQLContext(TestSQLContext.sparkContext) ParquetTestData.writeFile() ParquetTestData.writeFilterFile() ParquetTestData.writeNestedFile1() @@ -718,11 +709,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection in addressbook") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir1.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir1.toString).toSchemaRDD data.registerTempTable("data") - val query = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM data") + val query = sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() assert(tmp.size === 2) assert(tmp(0).size === 2) @@ -733,21 +722,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Simple query on nested int data") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir2.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir2.toString).toSchemaRDD data.registerTempTable("data") - val result1 = nestedParserSqlContext.sql("SELECT entries[0].value FROM data").collect() + val result1 = sql("SELECT entries[0].value FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === 2.5) - val result2 = nestedParserSqlContext.sql("SELECT entries[0] FROM data").collect() + val result2 = sql("SELECT entries[0] FROM data").collect() assert(result2.size === 1) val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] assert(subresult1.size === 2) assert(subresult1(0) === 2.5) assert(subresult1(1) === false) - val result3 = nestedParserSqlContext.sql("SELECT outerouter FROM data").collect() + val result3 = sql("SELECT outerouter FROM data").collect() val subresult2 = result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] @@ -760,19 +747,18 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("nested structs") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir3.toString) + val data = parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD data.registerTempTable("data") - val result1 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() + val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === false) - val result2 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() + val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() assert(result2.size === 1) assert(result2(0).size === 1) assert(result2(0)(0) === true) - val result3 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() + val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() assert(result3.size === 1) assert(result3(0).size === 1) assert(result3(0)(0) === false) @@ -796,11 +782,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("map with struct values") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD data.registerTempTable("mapTable") - val result1 = nestedParserSqlContext.sql("SELECT data2 FROM mapTable").collect() + val result1 = sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) val entry1 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] @@ -814,7 +798,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result2 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() + val result2 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() assert(result2.size === 1) assert(result2(0)(0) === 42.toLong) assert(result2(0)(1) === "the answer") @@ -825,15 +809,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // has no effect in this test case val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) - val result = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir1.toString) - .toSchemaRDD + val result = parquetFile(ParquetTestData.testNestedDir1.toString).toSchemaRDD result.saveAsParquetFile(tmpdir.toString) - nestedParserSqlContext - .parquetFile(tmpdir.toString) + parquetFile(tmpdir.toString) .toSchemaRDD .registerTempTable("tmpcopy") - val tmpdata = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() + val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() assert(tmpdata.size === 2) assert(tmpdata(0).size === 2) assert(tmpdata(0)(0) === "Julien Le Dem") @@ -844,20 +825,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Writing out Map and reading it back in") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) data.saveAsParquetFile(tmpdir.toString) - nestedParserSqlContext - .parquetFile(tmpdir.toString) + parquetFile(tmpdir.toString) .toSchemaRDD .registerTempTable("tmpmapcopy") - val result1 = nestedParserSqlContext.sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() + val result1 = sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() assert(result1.size === 1) assert(result1(0)(0) === 2) - val result2 = nestedParserSqlContext.sql("SELECT data2 FROM tmpmapcopy").collect() + val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() assert(result2.size === 1) val entry1 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] @@ -871,42 +849,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result3 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() + val result3 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() assert(result3.size === 1) assert(result3(0)(0) === 42.toLong) assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } } - -// TODO: the code below is needed temporarily until the standard parser is able to parse -// nested field expressions correctly -class NestedParserSQLContext(@transient override val sparkContext: SparkContext) extends SQLContext(sparkContext) { - override protected[sql] val parser = new NestedSqlParser() -} - -class NestedSqlLexical(override val keywords: Seq[String]) extends SqlLexical(keywords) { - override def identChar = letter | elem('_') - delimiters += (".") -} - -class NestedSqlParser extends SqlParser { - override val lexical = new NestedSqlLexical(reservedWords) - - override protected lazy val baseExpression: PackratParser[Expression] = - expression ~ "[" ~ expression <~ "]" ^^ { - case base ~ _ ~ ordinal => GetItem(base, ordinal) - } | - expression ~ "." ~ ident ^^ { - case base ~ _ ~ fieldName => GetField(base, fieldName) - } | - TRUE ^^^ Literal(true, BooleanType) | - FALSE ^^^ Literal(false, BooleanType) | - cast | - "(" ~> expression <~ ")" | - function | - "-" ~> literal ^^ UnaryMinus | - ident ^^ UnresolvedAttribute | - "*" ^^^ Star(None) | - literal -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 635a9fb0d56cb..b99caf77bce28 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.hive.execution -import scala.reflect.ClassTag - -import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHive._ +case class Nested1(f1: Nested2) +case class Nested2(f2: Nested3) +case class Nested3(f3: Int) + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -47,4 +47,11 @@ class SQLQuerySuite extends QueryTest { GROUP BY key, value ORDER BY value) a""").collect().toSeq) } + + test("double nested data") { + sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil).registerTempTable("nested") + checkAnswer( + sql("SELECT f1.f2.f3 FROM nested"), + 1) + } } From 558962a83fb0758ab5c13ff4ea58cc96c29cbbcc Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Wed, 10 Sep 2014 13:06:47 -0700 Subject: [PATCH 31/76] [SPARK-3411] Improve load-balancing of concurrently-submitted drivers across workers If the waiting driver array is too big, the drivers in it will be dispatched to the first worker we get(if it has enough resources), with or without the Randomization. We should do randomization every time we dispatch a driver, in order to better balance drivers. Author: WangTaoTheTonic Author: WangTao Closes #1106 from WangTaoTheTonic/fixBalanceDrivers and squashes the following commits: d1a928b [WangTaoTheTonic] Minor adjustment b6560cf [WangTaoTheTonic] solve the shuffle problem for HashSet f674e59 [WangTaoTheTonic] add comment and minor fix 2835929 [WangTao] solve the failed test and avoid filtering 2ca3091 [WangTao] fix checkstyle bc91bb1 [WangTao] Avoid shuffle every time we schedule the driver using round robin bbc7087 [WangTaoTheTonic] Optimize the schedule in Master --- .../apache/spark/deploy/master/Master.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index a3909d6ea95c0..2a3bd6ba0b9dc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -487,13 +487,25 @@ private[spark] class Master( if (state != RecoveryState.ALIVE) { return } // First schedule drivers, they take strict precedence over applications - val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers - for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { - for (driver <- List(waitingDrivers: _*)) { // iterate over a copy of waitingDrivers + // Randomization helps balance drivers + val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) + val aliveWorkerNum = shuffledAliveWorkers.size + var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers + // We assign workers to each waiting driver in a round-robin fashion. For each driver, we + // start from the last worker that was assigned a driver, and continue onwards until we have + // explored all alive workers. + curPos = (curPos + 1) % aliveWorkerNum + val startPos = curPos + var launched = false + while (curPos != startPos && !launched) { + val worker = shuffledAliveWorkers(curPos) if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver + launched = true } + curPos = (curPos + 1) % aliveWorkerNum } } From 79cdb9b64ad2fa3ab7f2c221766d36658b917c40 Mon Sep 17 00:00:00 2001 From: "qiping.lqp" Date: Wed, 10 Sep 2014 15:37:10 -0700 Subject: [PATCH 32/76] [SPARK-2207][SPARK-3272][MLLib]Add minimum information gain and minimum instances per node as training parameters for decision tree. These two parameters can act as early stop rules to do pre-pruning. When a split cause cause left or right child to have less than `minInstancesPerNode` or has less information gain than `minInfoGain`, current node will not be split by this split. When there is no possible splits that satisfy requirements, there is no useful information gain stats, but we still need to calculate the predict value for current node. So I separated calculation of predict from calculation of information gain, which can also save computation when the number of possible splits is large. Please see [SPARK-3272](https://issues.apache.org/jira/browse/SPARK-3272) for more details. CC: mengxr manishamde jkbradley, please help me review this, thanks. Author: qiping.lqp Author: chouqin Closes #2332 from chouqin/dt-preprune and squashes the following commits: f1d11d1 [chouqin] fix typo c7ebaf1 [chouqin] fix typo 39f9b60 [chouqin] change edge `minInstancesPerNode` to 2 and add one more test 0278a11 [chouqin] remove `noSplit` and set `Predict` private to tree d593ec7 [chouqin] fix docs and change minInstancesPerNode to 1 efcc736 [qiping.lqp] fix bug 10b8012 [qiping.lqp] fix style 6728fad [qiping.lqp] minor fix: remove empty lines bb465ca [qiping.lqp] Merge branch 'master' of https://github.com/apache/spark into dt-preprune cadd569 [qiping.lqp] add api docs 46b891f [qiping.lqp] fix bug e72c7e4 [qiping.lqp] add comments 845c6fa [qiping.lqp] fix style f195e83 [qiping.lqp] fix style 987cbf4 [qiping.lqp] fix bug ff34845 [qiping.lqp] separate calculation of predict of node from calculation of info gain ac42378 [qiping.lqp] add min info gain and min instances per node parameters in decision tree --- .../spark/mllib/tree/DecisionTree.scala | 72 ++++++++---- .../mllib/tree/configuration/Strategy.scala | 9 ++ .../tree/impl/DecisionTreeMetadata.scala | 7 +- .../tree/model/InformationGainStats.scala | 20 ++-- .../spark/mllib/tree/model/Predict.scala | 36 ++++++ .../apache/spark/mllib/tree/model/Split.scala | 2 + .../spark/mllib/tree/DecisionTreeSuite.scala | 103 ++++++++++++++++-- 7 files changed, 213 insertions(+), 36 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index d1309b2b20f54..98596569b8c95 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -130,7 +130,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel: Array[(Split, InformationGainStats)] = + val splitsStatsForLevel: Array[(Split, InformationGainStats, Predict)] = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") @@ -143,8 +143,9 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo timer.start("extractNodeInfo") val split = nodeSplitStats._1 val stats = nodeSplitStats._2 + val predict = nodeSplitStats._3.predict val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) + val node = new Node(nodeIndex, predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) nodes(nodeIndex) = node timer.stop("extractNodeInfo") @@ -425,7 +426,7 @@ object DecisionTree extends Serializable with Logging { splits: Array[Array[Split]], bins: Array[Array[Bin]], maxLevelForSingleGroup: Int, - timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats)] = { + timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats, Predict)] = { // split into groups to avoid memory overflow during aggregation if (level > maxLevelForSingleGroup) { // When information for all nodes at a given level cannot be stored in memory, @@ -434,7 +435,7 @@ object DecisionTree extends Serializable with Logging { // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) - var bestSplits = new Array[(Split, InformationGainStats)](0) + var bestSplits = new Array[(Split, InformationGainStats, Predict)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 while (groupIndex < numGroups) { @@ -605,7 +606,7 @@ object DecisionTree extends Serializable with Logging { bins: Array[Array[Bin]], timer: TimeTracker, numGroups: Int = 1, - groupIndex: Int = 0): Array[(Split, InformationGainStats)] = { + groupIndex: Int = 0): Array[(Split, InformationGainStats, Predict)] = { /* * The high-level descriptions of the best split optimizations are noted here. @@ -705,7 +706,7 @@ object DecisionTree extends Serializable with Logging { // Calculate best splits for all nodes at a given level timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats)](numNodes) + val bestSplits = new Array[(Split, InformationGainStats, Predict)](numNodes) // Iterating over all nodes at this level var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -734,28 +735,27 @@ object DecisionTree extends Serializable with Logging { topImpurity: Double, level: Int, metadata: DecisionTreeMetadata): InformationGainStats = { - val leftCount = leftImpurityCalculator.count val rightCount = rightImpurityCalculator.count - val totalCount = leftCount + rightCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + // If left child or right child doesn't satisfy minimum instances per node, + // then this split is invalid, return invalid information gain stats. + if ((leftCount < metadata.minInstancesPerNode) || + (rightCount < metadata.minInstancesPerNode)) { + return InformationGainStats.invalidInformationGainStats } - val parentNodeAgg = leftImpurityCalculator.copy - parentNodeAgg.add(rightImpurityCalculator) + val totalCount = leftCount + rightCount + // impurity of parent node val impurity = if (level > 0) { topImpurity } else { + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) parentNodeAgg.calculate() } - val predict = parentNodeAgg.predict - val prob = parentNodeAgg.prob(predict) - val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 val rightImpurity = rightImpurityCalculator.calculate() @@ -764,7 +764,31 @@ object DecisionTree extends Serializable with Logging { val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + // if information gain doesn't satisfy minimum information gain, + // then this split is invalid, return invalid information gain stats. + if (gain < metadata.minInfoGain) { + return InformationGainStats.invalidInformationGainStats + } + + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity) + } + + /** + * Calculate predict value for current node, given stats of any split. + * Note that this function is called only once for each node. + * @param leftImpurityCalculator left node aggregates for a split + * @param rightImpurityCalculator right node aggregates for a node + * @return predict value for current node + */ + private def calculatePredict( + leftImpurityCalculator: ImpurityCalculator, + rightImpurityCalculator: ImpurityCalculator): Predict = { + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + val predict = parentNodeAgg.predict + val prob = parentNodeAgg.prob(predict) + + new Predict(predict, prob) } /** @@ -780,12 +804,15 @@ object DecisionTree extends Serializable with Logging { nodeImpurity: Double, level: Int, metadata: DecisionTreeMetadata, - splits: Array[Array[Split]]): (Split, InformationGainStats) = { + splits: Array[Array[Split]]): (Split, InformationGainStats, Predict) = { logDebug("node impurity = " + nodeImpurity) + // calculate predict only once + var predict: Option[Predict] = None + // For each (feature, split), calculate the gain, and select the best (feature, split). - Range(0, metadata.numFeatures).map { featureIndex => + val (bestSplit, bestSplitStats) = Range(0, metadata.numFeatures).map { featureIndex => val numSplits = metadata.numSplits(featureIndex) if (metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. @@ -803,6 +830,7 @@ object DecisionTree extends Serializable with Logging { val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) + predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) (splitIdx, gainStats) @@ -816,6 +844,7 @@ object DecisionTree extends Serializable with Logging { Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) + predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) (splitIndex, gainStats) @@ -887,6 +916,7 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) + predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) (splitIndex, gainStats) @@ -898,6 +928,10 @@ object DecisionTree extends Serializable with Logging { (bestFeatureSplit, bestFeatureGainStats) } }.maxBy(_._2.gain) + + require(predict.isDefined, "must calculate predict for each node") + + (bestSplit, bestSplitStats, predict.get) } /** 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 23f74d5360fe5..987fe632c91ed 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 @@ -49,6 +49,13 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * k) implies the feature n is categorical with k categories 0, * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. + * @param minInstancesPerNode Minimum number of instances each child must have after split. + * Default value is 1. If a split cause left or right child + * to have less than minInstancesPerNode, + * this split will not be considered as a valid split. + * @param minInfoGain Minimum information gain a split must get. Default value is 0.0. + * If a split has less information gain than minInfoGain, + * this split will not be considered as a valid split. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is * 256 MB. */ @@ -61,6 +68,8 @@ class Strategy ( val maxBins: Int = 32, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), + val minInstancesPerNode: Int = 1, + val minInfoGain: Double = 0.0, val maxMemoryInMB: Int = 256) extends Serializable { if (algo == Classification) { 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 e95add7558bcf..5ceaa8154d11a 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 @@ -45,7 +45,9 @@ private[tree] class DecisionTreeMetadata( val unorderedFeatures: Set[Int], val numBins: Array[Int], val impurity: Impurity, - val quantileStrategy: QuantileStrategy) extends Serializable { + val quantileStrategy: QuantileStrategy, + val minInstancesPerNode: Int, + val minInfoGain: Double) extends Serializable { def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) @@ -127,7 +129,8 @@ private[tree] object DecisionTreeMetadata { new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, - strategy.impurity, strategy.quantileCalculationStrategy) + strategy.impurity, strategy.quantileCalculationStrategy, + strategy.minInstancesPerNode, strategy.minInfoGain) } /** 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 fb12298e0f5d3..f3e2619bd8ba0 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 @@ -26,20 +26,26 @@ import org.apache.spark.annotation.DeveloperApi * @param impurity current node impurity * @param leftImpurity left node impurity * @param rightImpurity right node impurity - * @param predict predicted value - * @param prob probability of the label (classification only) */ @DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, val leftImpurity: Double, - val rightImpurity: Double, - val predict: Double, - val prob: Double = 0.0) extends Serializable { + val rightImpurity: Double) extends Serializable { override def toString = { - "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f, prob = %f" - .format(gain, impurity, leftImpurity, rightImpurity, predict, prob) + "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" + .format(gain, impurity, leftImpurity, rightImpurity) } } + + +private[tree] object InformationGainStats { + /** + * An [[org.apache.spark.mllib.tree.model.InformationGainStats]] object to + * denote that current split doesn't satisfies minimum info gain or + * minimum number of instances per node. + */ + val invalidInformationGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala new file mode 100644 index 0000000000000..6fac2be2797bc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -0,0 +1,36 @@ +/* + * 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 + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Predicted value for a node + * @param predict predicted value + * @param prob probability of the label (classification only) + */ +@DeveloperApi +private[tree] class Predict( + val predict: Double, + val prob: Double = 0.0) extends Serializable{ + + override def toString = { + "predict = %f, prob = %f".format(predict, prob) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 50fb48b40de3d..b7a85f58544a3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType +import org.apache.spark.mllib.tree.configuration.FeatureType +import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** * :: DeveloperApi :: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 69482f2acbb40..fd8547c1660fc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} +import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext class DecisionTreeSuite extends FunSuite with LocalSparkContext { @@ -279,9 +279,10 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(split.threshold === Double.MinValue) val stats = bestSplits(0)._2 + val predict = bestSplits(0)._3 assert(stats.gain > 0) - assert(stats.predict === 1) - assert(stats.prob === 0.6) + assert(predict.predict === 1) + assert(predict.prob === 0.6) assert(stats.impurity > 0.2) } @@ -312,8 +313,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(split.threshold === Double.MinValue) val stats = bestSplits(0)._2 + val predict = bestSplits(0)._3.predict assert(stats.gain > 0) - assert(stats.predict === 0.6) + assert(predict === 0.6) assert(stats.impurity > 0.2) } @@ -387,7 +389,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._2.predict === 1) + assert(bestSplits(0)._3.predict === 1) } test("Binary classification stump with fixed label 0 for Entropy") { @@ -414,7 +416,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._2.predict === 0) + assert(bestSplits(0)._3.predict === 0) } test("Binary classification stump with fixed label 1 for Entropy") { @@ -441,7 +443,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._2.predict === 1) + assert(bestSplits(0)._3.predict === 1) } test("Second level node building with vs. without groups") { @@ -490,7 +492,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(i)._2.impurity === bestSplitsWithGroups(i)._2.impurity) assert(bestSplits(i)._2.leftImpurity === bestSplitsWithGroups(i)._2.leftImpurity) assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) - assert(bestSplits(i)._2.predict === bestSplitsWithGroups(i)._2.predict) + assert(bestSplits(i)._3.predict === bestSplitsWithGroups(i)._3.predict) } } @@ -674,6 +676,91 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { validateClassifier(model, arr, 0.6) } + test("split must satisfy min instances per node requirements") { + val arr = new Array[LabeledPoint](3) + arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) + arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, + maxDepth = 2, numClassesForClassification = 2, minInstancesPerNode = 2) + + val model = DecisionTree.train(input, strategy) + assert(model.topNode.isLeaf) + assert(model.topNode.predict == 0.0) + val predicts = input.map(p => model.predict(p.features)).collect() + predicts.foreach { predict => + assert(predict == 0.0) + } + + // test for findBestSplits when no valid split can be found + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length == 1) + val bestInfoStats = bestSplits(0)._2 + assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + } + + test("don't choose split that doesn't satisfy min instance per node requirements") { + // if a split doesn't satisfy min instances per node requirements, + // this split is invalid, even though the information gain of split is large. + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0, 1.0)) + arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, + maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), + numClassesForClassification = 2, minInstancesPerNode = 2) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length == 1) + val bestSplit = bestSplits(0)._1 + val bestSplitStats = bestSplits(0)._1 + assert(bestSplit.feature == 1) + assert(bestSplitStats != InformationGainStats.invalidInformationGainStats) + } + + test("split must satisfy min info gain requirements") { + val arr = new Array[LabeledPoint](3) + arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) + arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, minInfoGain = 1.0) + + val model = DecisionTree.train(input, strategy) + assert(model.topNode.isLeaf) + assert(model.topNode.predict == 0.0) + val predicts = input.map(p => model.predict(p.features)).collect() + predicts.foreach { predict => + assert(predict == 0.0) + } + + // test for findBestSplits when no valid split can be found + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length == 1) + val bestInfoStats = bestSplits(0)._2 + assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + } } object DecisionTreeSuite { From 84e2c8bfe41837baf2aeffa9741e4dbd14351981 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 10 Sep 2014 20:57:38 -0700 Subject: [PATCH 33/76] [SQL] Add test case with workaround for reading partitioned Avro files In order to read from partitioned Avro files we need to also set the `SERDEPROPERTIES` since `TBLPROPERTIES` are not passed to the initialization. This PR simply adds a test to make sure we don't break this workaround. Author: Michael Armbrust Closes #2340 from marmbrus/avroPartitioned and squashes the following commits: 6b969d6 [Michael Armbrust] fix style fea2124 [Michael Armbrust] Add test case with workaround for reading partitioned avro files. --- .../org/apache/spark/sql/hive/TestHive.scala | 69 ++++++++++++++++++- ...roSerDe-0-e4501461c855cc9071a872a64186c3de | 8 +++ .../sql/hive/execution/HiveSerDeSuite.scala | 2 + 3 files changed, 78 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a013f3f7a805f..6974f3e581b97 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -269,7 +269,74 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { |) """.stripMargin.cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' INTO TABLE episodes".cmd - ) + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC PARITIONING + // IS NOT YET SUPPORTED + TestTable("episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' + |STORED AS + |INPUTFORMAT '${classOf[AvroContainerInputFormat].getCanonicalName}' + |OUTPUTFORMAT '${classOf[AvroContainerOutputFormat].getCanonicalName}' + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ) ) hiveQTestUtilTables.foreach(registerTestTable) diff --git a/sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de b/sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de new file mode 100644 index 0000000000000..49c8434730ffa --- /dev/null +++ b/sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de @@ -0,0 +1,8 @@ +The Eleventh Hour 3 April 2010 11 1 +The Doctor's Wife 14 May 2011 11 1 +Horror of Fang Rock 3 September 1977 4 1 +An Unearthly Child 23 November 1963 1 1 +The Mysterious Planet 6 September 1986 6 1 +Rose 26 March 2005 9 1 +The Power of the Daleks 5 November 1966 2 1 +Castrolava 4 January 1982 5 1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 8bc72384a64ee..7486bfa82b00b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -37,4 +37,6 @@ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") + + createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") } From f92cde24e8f30dddd5bcec71bb3687498c1406da Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 10 Sep 2014 20:59:40 -0700 Subject: [PATCH 34/76] [SPARK-3447][SQL] Remove explicit conversion with JListWrapper to avoid NPE Author: Michael Armbrust Closes #2323 from marmbrus/kryoJListNPE and squashes the following commits: 9634f11 [Michael Armbrust] Rollback JSON RDD changes 4d4d93c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into kryoJListNPE 646976b [Michael Armbrust] Fix JSON RDD Conversion too 59065bc [Michael Armbrust] Remove explicit conversion to avoid NPE --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a2f334aab9fdf..c551c7c9877e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -460,7 +460,6 @@ class SQLContext(@transient val sparkContext: SparkContext) rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { import scala.collection.JavaConversions._ - import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} def needsConversion(dataType: DataType): Boolean = dataType match { case ByteType => true @@ -482,8 +481,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case (null, _) => null case (c: java.util.List[_], ArrayType(elementType, _)) => - val converted = c.map { e => convert(e, elementType)} - JListWrapper(converted) + c.map { e => convert(e, elementType)}: Seq[Any] case (c, ArrayType(elementType, _)) if c.getClass.isArray => c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] From c27718f376483dbe6290de612094c8d4ce9b16b4 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Wed, 10 Sep 2014 21:01:53 -0700 Subject: [PATCH 35/76] [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit LogicalPlan contains a ‘resolved’ attribute indicating that all of its execution requirements have been resolved. This attribute is not checked before query execution. The analyzer contains a step to check that all Expressions are resolved, but this is not equivalent to checking all LogicalPlans. In particular, the Union plan’s implementation of ‘resolved’ verifies that the types of its children’s columns are compatible. Because the analyzer does not check that a Union plan is resolved, it is possible to execute a Union plan that outputs different types in the same column. See SPARK-2781 for an example. This patch adds two checks to the analyzer’s CheckResolution rule. First, each logical plan is checked to see if it is not resolved despite its children being resolved. This allows the ‘problem’ unresolved plan to be included in the TreeNodeException for reporting. Then as a backstop the root plan is checked to see if it is resolved, which recursively checks that the entire plan tree is resolved. Note that the resolved attribute is implemented recursively, and this patch also explicitly checks the resolved attribute on each logical plan in the tree. I assume the query plan trees will not be large enough for this redundant checking to meaningfully impact performance. Because this patch starts validating that LogicalPlans are resolved before execution, I had to fix some cases where unresolved plans were passing through the analyzer as part of the implementation of the hive query system. In particular, HiveContext applies the CreateTables and PreInsertionCasts, and ExtractPythonUdfs rules manually after the analyzer runs. I moved these rules to the analyzer stage (for hive queries only), in the process completing a code TODO indicating the rules should be moved to the analyzer. It’s worth noting that moving the CreateTables rule means introducing an analyzer rule with a significant side effect - in this case the side effect is creating a hive table. The rule will only attempt to create a table once even if its batch is executed multiple times, because it converts the InsertIntoCreatedTable plan it matches against into an InsertIntoTable. Additionally, these hive rules must be added to the Resolution batch rather than as a separate batch because hive rules rules may be needed to resolve non-root nodes, leaving the root to be resolved on a subsequent batch iteration. For example, the hive compatibility test auto_smb_mapjoin_14, and others, make use of a query plan where the root is a Union and its children are each a hive InsertIntoTable. Mixing the custom hive rules with standard analyzer rules initially resulted in an additional failure because of policy differences between spark sql and hive when casting a boolean to a string. Hive casts booleans to strings as “true” / “false” while spark sql casts booleans to strings as “1” / “0” (causing the cast1.q test to fail). This behavior is a result of the BooleanCasts rule in HiveTypeCoercion.scala, and from looking at the implementation of BooleanCasts I think converting to to “1”/“0” is potentially a programming mistake. (If the BooleanCasts rule is disabled, casting produces “true”/“false” instead.) I believe “true” / “false” should be the behavior for spark sql - I changed the behavior so bools are converted to “true”/“false” to be consistent with hive, and none of the existing spark tests failed. Finally, in some initial testing with hive it appears that an implicit type coercion of boolean to string results in a lowercase string, e.g. CONCAT( TRUE, “” ) -> “true” while an explicit cast produces an all caps string, e.g. CAST( TRUE AS STRING ) -> “TRUE”. The change I’ve made just converts to lowercase strings in all cases. I believe it is at least more correct than the existing spark sql implementation where all Cast expressions become “1” / “0”. Author: Aaron Staple Closes #1706 from staple/SPARK-2781 and squashes the following commits: 32683c4 [Aaron Staple] Fix compilation failure due to merge. 7c77fda [Aaron Staple] Move ExtractPythonUdfs to Analyzer's extendedRules in HiveContext. d49bfb3 [Aaron Staple] Address review comments. 915b690 [Aaron Staple] Fix merge issue causing compilation failure. 701dcd2 [Aaron Staple] [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer. --- .../sql/catalyst/analysis/Analyzer.scala | 21 +++++++-- .../catalyst/analysis/HiveTypeCoercion.scala | 4 ++ .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 13 +++++- .../analysis/HiveTypeCoercionSuite.scala | 15 +++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 45 +++++++++++++++++-- .../apache/spark/sql/hive/HiveContext.scala | 11 +++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 16 ++++--- 8 files changed, 107 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4a9524074132e..574d96d92942b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -40,7 +40,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) - val batches: Seq[Batch] = Seq( + /** + * Override to provide additional rules for the "Resolution" batch. + */ + val extendedRules: Seq[Rule[LogicalPlan]] = Nil + + lazy val batches: Seq[Batch] = Seq( Batch("MultiInstanceRelations", Once, NewRelationInstances), Batch("CaseInsensitiveAttributeReferences", Once, @@ -54,8 +59,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool StarExpansion :: ResolveFunctions :: GlobalAggregates :: - UnresolvedHavingClauseAttributes :: - typeCoercionRules :_*), + UnresolvedHavingClauseAttributes :: + typeCoercionRules ++ + extendedRules : _*), Batch("Check Analysis", Once, CheckResolution), Batch("AnalysisOperators", fixedPoint, @@ -63,7 +69,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) /** - * Makes sure all attributes have been resolved. + * Makes sure all attributes and logical plans have been resolved. */ object CheckResolution extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { @@ -71,6 +77,13 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case p if p.expressions.exists(!_.resolved) => throw new TreeNodeException(p, s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}") + case p if !p.resolved && p.childrenResolved => + throw new TreeNodeException(p, "Unresolved plan found") + } match { + // As a backstop, use the root node to check that the entire plan tree is resolved. + case p if !p.resolved => + throw new TreeNodeException(p, "Unresolved plan in tree") + case p => p } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index bd8131c9af6e0..79e5283e86a37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -286,6 +286,10 @@ trait HiveTypeCoercion { // If the data type is not boolean and is being cast boolean, turn it into a comparison // with the numeric value, i.e. x != 0. This will coerce the type into numeric type. case Cast(e, BooleanType) if e.dataType != BooleanType => Not(EqualTo(e, Literal(0))) + // Stringify boolean if casting to StringType. + // TODO Ensure true/false string letter casing is consistent with Hive in all cases. + case Cast(e, StringType) if e.dataType == BooleanType => + If(e, Literal("true"), Literal("false")) // Turn true into 1, and false into 0 if casting boolean into other types. case Cast(e, dataType) if e.dataType == BooleanType => Cast(If(e, Literal(1), Literal(0)), dataType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index bae491f07c13f..ede431ad4ab27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -58,7 +58,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { /** * Returns true if this expression and all its children have been resolved to a specific schema - * and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan + * and false if it still contains any unresolved placeholders. Implementations of LogicalPlan * can override this (e.g. * [[org.apache.spark.sql.catalyst.analysis.UnresolvedRelation UnresolvedRelation]] * should return `false`). diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 0a4fde3de7752..5809a108ff62e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -93,6 +93,17 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { val e = intercept[TreeNodeException[_]] { caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) } - assert(e.getMessage().toLowerCase.contains("unresolved")) + assert(e.getMessage().toLowerCase.contains("unresolved attribute")) + } + + test("throw errors for unresolved plans during analysis") { + case class UnresolvedTestPlan() extends LeafNode { + override lazy val resolved = false + override def output = Nil + } + val e = intercept[TreeNodeException[_]] { + caseSensitiveAnalyze(UnresolvedTestPlan()) + } + assert(e.getMessage().toLowerCase.contains("unresolved plan")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index ba8b853b6f99e..baeb9b0cf5964 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.types._ class HiveTypeCoercionSuite extends FunSuite { @@ -84,4 +86,17 @@ class HiveTypeCoercionSuite extends FunSuite { widenTest(StringType, MapType(IntegerType, StringType, true), None) widenTest(ArrayType(IntegerType), StructType(Seq()), None) } + + test("boolean casts") { + val booleanCasts = new HiveTypeCoercion { }.BooleanCasts + def ruleTest(initial: Expression, transformed: Expression) { + val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) + assert(booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)) == + Project(Seq(Alias(transformed, "a")()), testRelation)) + } + // Remove superflous boolean -> boolean casts. + ruleTest(Cast(Literal(true), BooleanType), Literal(true)) + // Stringify boolean when casting to string. + ruleTest(Cast(Literal(false), StringType), If(Literal(false), Literal("true"), Literal("false"))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 514ac543df92a..67563b6c55f4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll @@ -477,18 +478,48 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null))) } - test("EXCEPT") { + test("UNION") { + checkAnswer( + sql("SELECT * FROM lowerCaseData UNION SELECT * FROM upperCaseData"), + (1, "A") :: (1, "a") :: (2, "B") :: (2, "b") :: (3, "C") :: (3, "c") :: + (4, "D") :: (4, "d") :: (5, "E") :: (6, "F") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData UNION SELECT * FROM lowerCaseData"), + (1, "a") :: (2, "b") :: (3, "c") :: (4, "d") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData UNION ALL SELECT * FROM lowerCaseData"), + (1, "a") :: (1, "a") :: (2, "b") :: (2, "b") :: (3, "c") :: (3, "c") :: + (4, "d") :: (4, "d") :: Nil) + } + test("UNION with column mismatches") { + // Column name mismatches are allowed. + checkAnswer( + sql("SELECT n,l FROM lowerCaseData UNION SELECT N as x1, L as x2 FROM upperCaseData"), + (1, "A") :: (1, "a") :: (2, "B") :: (2, "b") :: (3, "C") :: (3, "c") :: + (4, "D") :: (4, "d") :: (5, "E") :: (6, "F") :: Nil) + // Column type mismatches are not allowed, forcing a type coercion. checkAnswer( - sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData "), + sql("SELECT n FROM lowerCaseData UNION SELECT L FROM upperCaseData"), + ("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Tuple1(_))) + // Column type mismatches where a coercion is not possible, in this case between integer + // and array types, trigger a TreeNodeException. + intercept[TreeNodeException[_]] { + sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect() + } + } + + test("EXCEPT") { + checkAnswer( + sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData"), (1, "a") :: (2, "b") :: (3, "c") :: (4, "d") :: Nil) checkAnswer( - sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData "), Nil) + sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData"), Nil) checkAnswer( - sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData "), Nil) + sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData"), Nil) } test("INTERSECT") { @@ -634,6 +665,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), Seq() ) + } + test("cast boolean to string") { + // TODO Ensure true/false string letter casing is consistent with Hive in all cases. + checkAnswer( + sql("SELECT CAST(TRUE AS STRING), CAST(FALSE AS STRING) FROM testData LIMIT 1"), + ("true", "false") :: Nil) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ced8397972fbd..e0be09e6793ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -262,7 +262,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* An analyzer that uses the Hive metastore. */ @transient override protected[sql] lazy val analyzer = - new Analyzer(catalog, functionRegistry, caseSensitive = false) + new Analyzer(catalog, functionRegistry, caseSensitive = false) { + override val extendedRules = + catalog.CreateTables :: + catalog.PreInsertionCasts :: + ExtractPythonUdfs :: + Nil + } /** * Runs the specified SQL query using Hive. @@ -353,9 +359,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** Extends QueryExecution with hive specific features. */ protected[sql] abstract class QueryExecution extends super.QueryExecution { - // TODO: Create mixin for the analyzer instead of overriding things here. - override lazy val optimizedPlan = - optimizer(ExtractPythonUdfs(catalog.PreInsertionCasts(catalog.CreateTables(analyzed)))) override lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 6571c35499ef4..dfa2a7a9d28bb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -109,6 +109,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Wait until children are resolved. + case p: LogicalPlan if !p.childrenResolved => p + case InsertIntoCreatedTable(db, tableName, child) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) @@ -116,8 +119,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with createTable(databaseName, tblName, child.output) InsertIntoTable( - EliminateAnalysisOperators( - lookupRelation(Some(databaseName), tblName, None)), + lookupRelation(Some(databaseName), tblName, None), Map.empty, child, overwrite = false) @@ -130,15 +132,17 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ object PreInsertionCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.transform { - // Wait until children are resolved + // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => + case p @ InsertIntoTable( + LowerCaseSchema(table: MetastoreRelation), _, child, _) => castChildOutput(p, table, child) case p @ logical.InsertIntoTable( - InMemoryRelation(_, _, _, - HiveTableScan(_, table, _)), _, child, _) => + LowerCaseSchema( + InMemoryRelation(_, _, _, + HiveTableScan(_, table, _))), _, child, _) => castChildOutput(p, table, child) } From ed1980ffa9ccb87d76694ba910ef22df034bca49 Mon Sep 17 00:00:00 2001 From: Chris Cope Date: Thu, 11 Sep 2014 08:13:07 -0500 Subject: [PATCH 36/76] [SPARK-2140] Updating heap memory calculation for YARN stable and alpha. Updated pull request, reflecting YARN stable and alpha states. I am getting intermittent test failures on my own test infrastructure. Is that tracked anywhere yet? Author: Chris Cope Closes #2253 from copester/master and squashes the following commits: 5ad89da [Chris Cope] [SPARK-2140] Removing calculateAMMemory functions since they are no longer needed. 52b4e45 [Chris Cope] [SPARK-2140] Updating heap memory calculation for YARN stable and alpha. --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 8 -------- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 4 +--- .../org/apache/spark/deploy/yarn/ClientBaseSuite.scala | 3 --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 9 --------- 4 files changed, 1 insertion(+), 23 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 10fc39bba87d1..aff9ab71f0937 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -103,14 +103,6 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa appContext } - def calculateAMMemory(newApp: GetNewApplicationResponse): Int = { - val minResMemory = newApp.getMinimumResourceCapability().getMemory() - val amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - memoryOverhead) - amMemory - } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { // Setup security tokens. val dob = new DataOutputBuffer() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8075b7a7fb837..c96f731923d22 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -300,8 +300,6 @@ trait ClientBase extends Logging { retval.toString } - def calculateAMMemory(newApp: GetNewApplicationResponse): Int - def setupSecurityToken(amContainer: ContainerLaunchContext) def createContainerLaunchContext( @@ -346,7 +344,7 @@ trait ClientBase extends Logging { } amContainer.setEnvironment(env) - val amMemory = calculateAMMemory(newApp) + val amMemory = args.amMemory val javaOpts = ListBuffer[String]() diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 68cc2890f3a22..5480eca7c832c 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -238,9 +238,6 @@ class ClientBaseSuite extends FunSuite with Matchers { val sparkConf: SparkConf, val yarnConf: YarnConfiguration) extends ClientBase { - override def calculateAMMemory(newApp: GetNewApplicationResponse): Int = - throw new UnsupportedOperationException() - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = throw new UnsupportedOperationException() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 313a0d21ce181..82e45e3e7ad54 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -103,15 +103,6 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa clusterMetrics.getNumNodeManagers) } - def calculateAMMemory(newApp: GetNewApplicationResponse) :Int = { - // TODO: Need a replacement for the following code to fix -Xmx? - // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() - // var amMemory = ((args.amMemory / minResMemory) * minResMemory) + - // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - // memoryOverhead ) - args.amMemory - } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { // Setup security tokens. val dob = new DataOutputBuffer() From 1ef656ea85b4b93c7b0f3cf8042b63a0de0901cb Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 11 Sep 2014 11:50:36 -0700 Subject: [PATCH 37/76] [SPARK-3047] [PySpark] add an option to use str in textFileRDD str is much efficient than unicode (both CPU and memory), it'e better to use str in textFileRDD. In order to keep compatibility, use unicode by default. (Maybe change it in the future). use_unicode=True: daviesliudm:~/work/spark$ time python wc.py (u'./universe/spark/sql/core/target/java/org/apache/spark/sql/execution/ExplainCommand$.java', 7776) real 2m8.298s user 0m0.185s sys 0m0.064s use_unicode=False daviesliudm:~/work/spark$ time python wc.py ('./universe/spark/sql/core/target/java/org/apache/spark/sql/execution/ExplainCommand$.java', 7776) real 1m26.402s user 0m0.182s sys 0m0.062s We can see that it got 32% improvement! Author: Davies Liu Closes #1951 from davies/unicode and squashes the following commits: 8352d57 [Davies Liu] update version number a286f2f [Davies Liu] rollback loads() 85246e5 [Davies Liu] add docs for use_unicode a0295e1 [Davies Liu] add an option to use str in textFile() --- python/pyspark/context.py | 16 ++++++++++++---- python/pyspark/serializers.py | 18 +++++++++++------- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 84bc0a3b7ccd0..3ab98e262df31 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -331,12 +331,16 @@ def pickleFile(self, name, minPartitions=None): return RDD(self._jsc.objectFile(name, minPartitions), self, BatchedSerializer(PickleSerializer())) - def textFile(self, name, minPartitions=None): + def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings. + If use_unicode is False, the strings will be kept as `str` (encoding + as `utf-8`), which is faster and smaller than unicode. (Added in + Spark 1.2) + >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: ... testFile.write("Hello world!") @@ -346,9 +350,9 @@ def textFile(self, name, minPartitions=None): """ minPartitions = minPartitions or min(self.defaultParallelism, 2) return RDD(self._jsc.textFile(name, minPartitions), self, - UTF8Deserializer()) + UTF8Deserializer(use_unicode)) - def wholeTextFiles(self, path, minPartitions=None): + def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system @@ -356,6 +360,10 @@ def wholeTextFiles(self, path, minPartitions=None): key-value pair, where the key is the path of each file, the value is the content of each file. + If use_unicode is False, the strings will be kept as `str` (encoding + as `utf-8`), which is faster and smaller than unicode. (Added in + Spark 1.2) + For example, if you have the following files:: hdfs://a-hdfs-path/part-00000 @@ -386,7 +394,7 @@ def wholeTextFiles(self, path, minPartitions=None): """ minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.wholeTextFiles(path, minPartitions), self, - PairDeserializer(UTF8Deserializer(), UTF8Deserializer())) + PairDeserializer(UTF8Deserializer(use_unicode), UTF8Deserializer(use_unicode))) def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 55e6cf3308611..7b2710b913128 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -429,18 +429,22 @@ class UTF8Deserializer(Serializer): Deserializes streams written by String.getBytes. """ + def __init__(self, use_unicode=False): + self.use_unicode = use_unicode + def loads(self, stream): length = read_int(stream) - return stream.read(length).decode('utf8') + s = stream.read(length) + return s.decode("utf-8") if self.use_unicode else s def load_stream(self, stream): - while True: - try: + try: + while True: yield self.loads(stream) - except struct.error: - return - except EOFError: - return + except struct.error: + return + except EOFError: + return def read_long(stream): From ca83f1e2c4dfa519e44b837b6815cba3b4526d92 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 11 Sep 2014 11:57:01 -0700 Subject: [PATCH 38/76] [SPARK-2917] [SQL] Avoid table creation in logical plan analyzing for CTAS Author: Cheng Hao Closes #1846 from chenghao-intel/ctas and squashes the following commits: 56a0578 [Cheng Hao] remove the unused imports 9a57abc [Cheng Hao] Avoid table creation in logical plan analyzing --- .../plans/logical/basicOperators.scala | 3 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 +--- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 10 +++ .../hive/execution/CreateTableAsSelect.scala | 73 +++++++++++++++++++ .../hive/execution/InsertIntoHiveTable.scala | 6 +- .../sql/hive/execution/SQLQuerySuite.scala | 9 +++ 8 files changed, 104 insertions(+), 17 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 4adfb189372d6..5d10754c7b028 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -114,11 +114,12 @@ case class InsertIntoTable( } } -case class InsertIntoCreatedTable( +case class CreateTableAsSelect( databaseName: Option[String], tableName: String, child: LogicalPlan) extends UnaryNode { override def output = child.output + override lazy val resolved = (databaseName != None && childrenResolved) } case class WriteToFile( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 2f3033a5f94f0..e52eeb3e1c47e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -54,7 +54,7 @@ private[sql] trait SchemaRDDLike { @transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: InsertIntoCreatedTable | _: WriteToFile => + case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => queryExecution.toRdd SparkLogicalPlan(queryExecution.executedPlan)(sqlContext) case _ => @@ -124,7 +124,7 @@ private[sql] trait SchemaRDDLike { */ @Experimental def saveAsTable(tableName: String): Unit = - sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd + sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan)).toRdd /** Returns the schema as a string in the tree format. * diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index dfa2a7a9d28bb..2c0db9be57e54 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -54,8 +54,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with db: Option[String], tableName: String, alias: Option[String]): LogicalPlan = synchronized { - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + val (databaseName, tblName) = processDatabaseAndTableName( + db.getOrElse(hive.sessionState.getCurrentDatabase), tableName) val table = client.getTable(databaseName, tblName) val partitions: Seq[Partition] = if (table.isPartitioned) { @@ -112,17 +112,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case InsertIntoCreatedTable(db, tableName, child) => + case CreateTableAsSelect(db, tableName, child) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - createTable(databaseName, tblName, child.output) - - InsertIntoTable( - lookupRelation(Some(databaseName), tblName, None), - Map.empty, - child, - overwrite = false) + CreateTableAsSelect(Some(databaseName), tableName, child) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index c98287c6aa662..21ecf17028dbc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -489,7 +489,7 @@ private[hive] object HiveQl { val (db, tableName) = extractDbNameTableName(tableNameParts) - InsertIntoCreatedTable(db, tableName, nodeToPlan(query)) + CreateTableAsSelect(db, tableName, nodeToPlan(query)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 72cc01cdf4c84..43dd3d234f73a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -165,6 +165,16 @@ private[hive] trait HiveStrategies { InMemoryRelation(_, _, _, HiveTableScan(_, table, _)), partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.CreateTableAsSelect(database, tableName, child) => + val query = planLater(child) + CreateTableAsSelect( + database.get, + tableName, + query, + InsertIntoHiveTable(_: MetastoreRelation, + Map(), + query, + true)(hiveContext)) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala new file mode 100644 index 0000000000000..71ea774d77795 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.annotation.Experimental +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LowerCaseSchema +import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.MetastoreRelation + +/** + * :: Experimental :: + * Create table and insert the query result into it. + * @param database the database name of the new relation + * @param tableName the table name of the new relation + * @param insertIntoRelation function of creating the `InsertIntoHiveTable` + * by specifying the `MetaStoreRelation`, the data will be inserted into that table. + * TODO Add more table creating properties, e.g. SerDe, StorageHandler, in-memory cache etc. + */ +@Experimental +case class CreateTableAsSelect( + database: String, + tableName: String, + query: SparkPlan, + insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) + extends LeafNode with Command { + + def output = Seq.empty + + // A lazy computing of the metastoreRelation + private[this] lazy val metastoreRelation: MetastoreRelation = { + // Create the table + val sc = sqlContext.asInstanceOf[HiveContext] + sc.catalog.createTable(database, tableName, query.output, false) + // Get the Metastore Relation + sc.catalog.lookupRelation(Some(database), tableName, None) match { + case LowerCaseSchema(r: MetastoreRelation) => r + case o: MetastoreRelation => o + } + } + + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + insertIntoRelation(metastoreRelation).execute + Seq.empty[Row] + } + + override def execute(): RDD[Row] = { + sideEffectResult + sparkContext.emptyRDD[Row] + } + + override def argString: String = { + s"[Database:$database, TableName: $tableName, InsertIntoHiveTable]\n" + query.toString + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 39033bdeac4b0..a284a91a91e31 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -53,9 +53,9 @@ case class InsertIntoHiveTable( (@transient sc: HiveContext) extends UnaryNode { - val outputClass = newSerializer(table.tableDesc).getSerializedClass - @transient private val hiveContext = new Context(sc.hiveconf) - @transient private val db = Hive.get(sc.hiveconf) + @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass + @transient private lazy val hiveContext = new Context(sc.hiveconf) + @transient private lazy val db = Hive.get(sc.hiveconf) private def newSerializer(tableDesc: TableDesc): Serializer = { val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index b99caf77bce28..679efe082f2a0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.QueryTest + +import org.apache.spark.sql.Row import org.apache.spark.sql.hive.test.TestHive._ case class Nested1(f1: Nested2) @@ -54,4 +56,11 @@ class SQLQuerySuite extends QueryTest { sql("SELECT f1.f2.f3 FROM nested"), 1) } + + test("test CTAS") { + checkAnswer(sql("CREATE TABLE test_ctas_123 AS SELECT key, value FROM src"), Seq.empty[Row]) + checkAnswer( + sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), + sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) + } } From 4bc9e046cb8922923dff254e3e621fb4de656f98 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 11 Sep 2014 15:23:33 -0700 Subject: [PATCH 39/76] [SPARK-3390][SQL] sqlContext.jsonRDD fails on a complex structure of JSON array and JSON object nesting This PR aims to correctly handle JSON arrays in the type of `ArrayType(...(ArrayType(StructType)))`. JIRA: https://issues.apache.org/jira/browse/SPARK-3390. Author: Yin Huai Closes #2364 from yhuai/SPARK-3390 and squashes the following commits: 46db418 [Yin Huai] Handle JSON arrays in the type of ArrayType(...(ArrayType(StructType))). --- .../org/apache/spark/sql/json/JsonRDD.scala | 66 +++++++++++-------- .../org/apache/spark/sql/json/JsonSuite.scala | 29 +++++++- .../apache/spark/sql/json/TestJsonData.scala | 30 ++++++++- 3 files changed, 96 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 70062eae3b7ce..873221835daf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -68,8 +68,15 @@ private[sql] object JsonRDD extends Logging { val (topLevel, structLike) = values.partition(_.size == 1) val topLevelFields = topLevel.filter { name => resolved.get(prefix ++ name).get match { - case ArrayType(StructType(Nil), _) => false - case ArrayType(_, _) => true + case ArrayType(elementType, _) => { + def hasInnerStruct(t: DataType): Boolean = t match { + case s: StructType => false + case ArrayType(t1, _) => hasInnerStruct(t1) + case o => true + } + + hasInnerStruct(elementType) + } case struct: StructType => false case _ => true } @@ -84,7 +91,18 @@ private[sql] object JsonRDD extends Logging { val dataType = resolved.get(prefix :+ name).get dataType match { case array: ArrayType => - Some(StructField(name, ArrayType(structType, array.containsNull), nullable = true)) + // The pattern of this array is ArrayType(...(ArrayType(StructType))). + // Since the inner struct of array is a placeholder (StructType(Nil)), + // we need to replace this placeholder with the actual StructType (structType). + def getActualArrayType( + innerStruct: StructType, + currentArray: ArrayType): ArrayType = currentArray match { + case ArrayType(s: StructType, containsNull) => + ArrayType(innerStruct, containsNull) + case ArrayType(a: ArrayType, containsNull) => + ArrayType(getActualArrayType(innerStruct, a), containsNull) + } + Some(StructField(name, getActualArrayType(structType, array), nullable = true)) case struct: StructType => Some(StructField(name, structType, nullable = true)) // dataType is StringType means that we have resolved type conflicts involving // primitive types and complex types. So, the type of name has been relaxed to @@ -168,8 +186,7 @@ private[sql] object JsonRDD extends Logging { /** * Returns the element type of an JSON array. We go through all elements of this array * to detect any possible type conflict. We use [[compatibleType]] to resolve - * type conflicts. Right now, when the element of an array is another array, we - * treat the element as String. + * type conflicts. */ private def typeOfArray(l: Seq[Any]): ArrayType = { val containsNull = l.exists(v => v == null) @@ -216,18 +233,24 @@ private[sql] object JsonRDD extends Logging { } case (key: String, array: Seq[_]) => { // The value associated with the key is an array. - typeOfArray(array) match { + // Handle inner structs of an array. + def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { case ArrayType(StructType(Nil), containsNull) => { // The elements of this arrays are structs. - array.asInstanceOf[Seq[Map[String, Any]]].flatMap { + v.asInstanceOf[Seq[Map[String, Any]]].flatMap { element => allKeysWithValueTypes(element) }.map { - case (k, dataType) => (s"$key.$k", dataType) - } :+ (key, ArrayType(StructType(Nil), containsNull)) + case (k, t) => (s"$key.$k", t) + } } - case ArrayType(elementType, containsNull) => - (key, ArrayType(elementType, containsNull)) :: Nil + case ArrayType(t1, containsNull) => + v.asInstanceOf[Seq[Any]].flatMap { + element => buildKeyPathForInnerStructs(element, t1) + } + case other => Nil } + val elementType = typeOfArray(array) + buildKeyPathForInnerStructs(array, elementType) :+ (key, elementType) } case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil } @@ -339,8 +362,6 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case ArrayType(elementType, _) => - value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case StringType => toString(value) case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) @@ -348,6 +369,10 @@ private[sql] object JsonRDD extends Logging { case DecimalType => toDecimal(value) case BooleanType => value.asInstanceOf[BooleanType.JvmType] case NullType => null + + case ArrayType(elementType, _) => + value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) + case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) } } } @@ -356,22 +381,9 @@ private[sql] object JsonRDD extends Logging { // TODO: Reuse the row instead of creating a new one for every record. val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { - // StructType - case (StructField(name, fields: StructType, _), i) => - row.update(i, json.get(name).flatMap(v => Option(v)).map( - v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) - - // ArrayType(StructType) - case (StructField(name, ArrayType(structType: StructType, _), _), i) => - row.update(i, - json.get(name).flatMap(v => Option(v)).map( - v => v.asInstanceOf[Seq[Any]].map( - e => asRow(e.asInstanceOf[Map[String, Any]], structType))).orNull) - - // Other cases case (StructField(name, dataType, _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( - enforceCorrectType(_, dataType)).getOrElse(null)) + enforceCorrectType(_, dataType)).orNull) } row diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 301d482d27d86..b50d93855405a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -591,8 +591,35 @@ class JsonSuite extends QueryTest { (true, "str1") :: Nil ) checkAnswer( - sql("select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] from jsonTable"), + sql( + """ + |select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] + |from jsonTable + """.stripMargin), ("str2", 6) :: Nil ) } + + test("SPARK-3390 Complex arrays") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType2) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql( + """ + |select arrayOfArray1[0][0][0], arrayOfArray1[1][0][1], arrayOfArray1[1][1][0] + |from jsonTable + """.stripMargin), + (5, 7, 8) :: Nil + ) + checkAnswer( + sql( + """ + |select arrayOfArray2[0][0][0].inner1, arrayOfArray2[1][0], + |arrayOfArray2[1][1][1].inner2[0], arrayOfArray2[2][0][0].inner3[0][0].inner4 + |from jsonTable + """.stripMargin), + ("str1", Nil, "str4", 2) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index b3f95f08e8044..5f0b3959a63ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -106,6 +106,34 @@ object TestJsonData { "inner1": "str4" }], "field2": [[5, 6], [7, 8]] - }] + }], + "arrayOfArray1": [ + [ + [5] + ], + [ + [6, 7], + [8] + ]], + "arrayOfArray2": [ + [ + [ + { + "inner1": "str1" + } + ] + ], + [ + [], + [ + {"inner2": ["str3", "str33"]}, + {"inner2": ["str4"], "inner1": "str11"} + ] + ], + [ + [ + {"inner3": [[{"inner4": 2}]]} + ] + ]] }""" :: Nil) } From 6324eb7b5b0ae005cb2e913e36b1508bd6f1b9b8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 11 Sep 2014 17:18:46 -0700 Subject: [PATCH 40/76] [Spark-3490] Disable SparkUI for tests We currently open many ephemeral ports during the tests, and as a result we occasionally can't bind to new ones. This has caused the `DriverSuite` and the `SparkSubmitSuite` to fail intermittently. By disabling the `SparkUI` when it's not needed, we already cut down on the number of ports opened significantly, on the order of the number of `SparkContexts` ever created. We must keep it enabled for a few tests for the UI itself, however. Author: Andrew Or Closes #2363 from andrewor14/disable-ui-for-tests and squashes the following commits: 332a7d5 [Andrew Or] No need to set spark.ui.port to 0 anymore 30c93a2 [Andrew Or] Simplify streaming UISuite a431b84 [Andrew Or] Fix streaming test failures 8f5ae53 [Andrew Or] Fix no new line at the end 29c9b5b [Andrew Or] Disable SparkUI for tests --- .../scala/org/apache/spark/SparkContext.scala | 12 +++-- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../cluster/SimrSchedulerBackend.scala | 6 +-- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../scala/org/apache/spark/ui/UISuite.scala | 44 +++++++++++++------ pom.xml | 2 +- project/SparkBuild.scala | 2 +- .../spark/streaming/StreamingContext.scala | 11 ++++- .../spark/streaming/StreamingSource.scala | 2 +- .../spark/streaming/ui/StreamingTab.scala | 25 ++++++++--- .../org/apache/spark/streaming/UISuite.scala | 16 +++++-- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 2 +- 13 files changed, 92 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c6c5b8f22b549..218b353dd9d49 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -220,8 +220,14 @@ class SparkContext(config: SparkConf) extends Logging { new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) // Initialize the Spark UI, registering all associated listeners - private[spark] val ui = new SparkUI(this) - ui.bind() + private[spark] val ui: Option[SparkUI] = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(new SparkUI(this)) + } else { + // For tests, do not enable the UI + None + } + ui.foreach(_.bind()) /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) @@ -990,7 +996,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { postApplicationEnd() - ui.stop() + ui.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 5b5257269d92f..9a0cb1c6c6ccd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -292,7 +292,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") conf.set("spark.ui.filters", filterName) conf.set(s"spark.$filterName.params", filterParams) - JettyUtils.addFilters(scheduler.sc.ui.getHandlers, conf) + scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index 513d74a08a47f..ee10aa061f4e9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.spark.{Logging, SparkContext, SparkEnv} @@ -47,16 +46,17 @@ private[spark] class SimrSchedulerBackend( val conf = SparkHadoopUtil.get.newConfiguration(sc.conf) val fs = FileSystem.get(conf) + val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") logInfo("Writing to HDFS file: " + driverFilePath) logInfo("Writing Akka address: " + driverUrl) - logInfo("Writing Spark UI Address: " + sc.ui.appUIAddress) + logInfo("Writing Spark UI Address: " + appUIAddress) // Create temporary file to prevent race condition where executors get empty driverUrl file val temp = fs.create(tmpPath, true) temp.writeUTF(driverUrl) temp.writeInt(maxCores) - temp.writeUTF(sc.ui.appUIAddress) + temp.writeUTF(appUIAddress) temp.close() // "Atomic" rename diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 06872ace2ecf4..2f45d192e1d4d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -67,8 +67,10 @@ private[spark] class SparkDeploySchedulerBackend( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) + val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") + val eventLogDir = sc.eventLogger.map(_.logDir) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) + appUIAddress, eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 038746d2eda4b..2f56642956701 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -36,11 +36,25 @@ import scala.xml.Node class UISuite extends FunSuite { + /** + * Create a test SparkContext with the SparkUI enabled. + * It is safe to `get` the SparkUI directly from the SparkContext returned here. + */ + private def newSparkContext(): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val sc = new SparkContext(conf) + assert(sc.ui.isDefined) + sc + } + ignore("basic ui visibility") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark(newSparkContext()) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.appUIAddress).mkString + val html = Source.fromURL(sc.ui.get.appUIAddress).mkString assert(!html.contains("random data that should not be present")) assert(html.toLowerCase.contains("stages")) assert(html.toLowerCase.contains("storage")) @@ -51,7 +65,7 @@ class UISuite extends FunSuite { } ignore("visibility at localhost:4040") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark(newSparkContext()) { sc => // test if visible from http://localhost:4040 eventually(timeout(10 seconds), interval(50 milliseconds)) { val html = Source.fromURL("http://localhost:4040").mkString @@ -61,8 +75,8 @@ class UISuite extends FunSuite { } ignore("attaching a new tab") { - withSpark(new SparkContext("local", "test")) { sc => - val sparkUI = sc.ui + withSpark(newSparkContext()) { sc => + val sparkUI = sc.ui.get val newTab = new WebUITab(sparkUI, "foo") { attachPage(new WebUIPage("") { @@ -73,7 +87,7 @@ class UISuite extends FunSuite { } sparkUI.attachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.appUIAddress).mkString + val html = Source.fromURL(sparkUI.appUIAddress).mkString assert(!html.contains("random data that should not be present")) // check whether new page exists @@ -87,7 +101,7 @@ class UISuite extends FunSuite { } eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString // check whether new page exists assert(html.contains("magic")) } @@ -129,16 +143,20 @@ class UISuite extends FunSuite { } test("verify appUIAddress contains the scheme") { - withSpark(new SparkContext("local", "test")) { sc => - val uiAddress = sc.ui.appUIAddress - assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + val uiAddress = ui.appUIAddress + val uiHostPort = ui.appUIHostPort + assert(uiAddress.equals("http://" + uiHostPort)) } } test("verify appUIAddress contains the port") { - withSpark(new SparkContext("local", "test")) { sc => - val splitUIAddress = sc.ui.appUIAddress.split(':') - assert(splitUIAddress(2).toInt == sc.ui.boundPort) + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + val splitUIAddress = ui.appUIAddress.split(':') + val boundPort = ui.boundPort + assert(splitUIAddress(2).toInt == boundPort) } } } diff --git a/pom.xml b/pom.xml index 64fb1e57e30e0..e5f863e85445c 100644 --- a/pom.xml +++ b/pom.xml @@ -899,7 +899,7 @@ true ${session.executionRootDirectory} 1 - 0 + false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 45f6d2973ea90..c07ea313f1228 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -337,7 +337,7 @@ object TestSettings { javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.ports.maxRetries=100", - javaOptions in Test += "-Dspark.ui.port=0", + javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 457e8ab28ed82..f63560dcb5b89 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -37,7 +37,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.ui.StreamingTab +import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} import org.apache.spark.util.MetadataCleaner /** @@ -158,7 +158,14 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter - private[streaming] val uiTab = new StreamingTab(this) + private[streaming] val progressListener = new StreamingJobProgressListener(this) + + private[streaming] val uiTab: Option[StreamingTab] = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(new StreamingTab(this)) + } else { + None + } /** Register streaming source to metrics system */ private val streamingSource = new StreamingSource(this) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index 75f0e8716dc7e..e35a568ddf115 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -26,7 +26,7 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { override val metricRegistry = new MetricRegistry override val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) - private val streamingListener = ssc.uiTab.listener + private val streamingListener = ssc.progressListener private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, defaultValue: T) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 34ac254f337eb..d9d04cd706a04 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -17,18 +17,31 @@ package org.apache.spark.streaming.ui -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} import org.apache.spark.streaming.StreamingContext -import org.apache.spark.ui.SparkUITab +import org.apache.spark.ui.{SparkUI, SparkUITab} -/** Spark Web UI tab that shows statistics of a streaming job */ +import StreamingTab._ + +/** + * Spark Web UI tab that shows statistics of a streaming job. + * This assumes the given SparkContext has enabled its SparkUI. + */ private[spark] class StreamingTab(ssc: StreamingContext) - extends SparkUITab(ssc.sc.ui, "streaming") with Logging { + extends SparkUITab(getSparkUI(ssc), "streaming") with Logging { - val parent = ssc.sc.ui - val listener = new StreamingJobProgressListener(ssc) + val parent = getSparkUI(ssc) + val listener = ssc.progressListener ssc.addStreamingListener(listener) attachPage(new StreamingPage(this)) parent.attachTab(this) } + +private object StreamingTab { + def getSparkUI(ssc: StreamingContext): SparkUI = { + ssc.sc.ui.getOrElse { + throw new SparkException("Parent SparkUI to attach this tab to not found!") + } + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 2a0db7564915d..4c7e43c2943c9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -24,13 +24,22 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ +import org.apache.spark.SparkConf + class UISuite extends FunSuite { // Ignored: See SPARK-1530 ignore("streaming tab in spark UI") { - val ssc = new StreamingContext("local", "test", Seconds(1)) + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val ssc = new StreamingContext(conf, Seconds(1)) + assert(ssc.sc.ui.isDefined, "Spark UI is not started!") + val ui = ssc.sc.ui.get + eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString + val html = Source.fromURL(ui.appUIAddress).mkString assert(!html.contains("random data that should not be present")) // test if streaming tab exist assert(html.toLowerCase.contains("streaming")) @@ -39,8 +48,7 @@ class UISuite extends FunSuite { } eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL( - ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + val html = Source.fromURL(ui.appUIAddress.stripSuffix("/") + "/streaming").mkString assert(html.toLowerCase.contains("batch")) assert(html.toLowerCase.contains("network")) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 5756263e89e21..878b6db546032 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -189,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIAddress, securityMgr) + registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) try { userThread.join() } finally { diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 41c662cd7a6de..6aa6475fe4a18 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -55,7 +55,7 @@ private[spark] class YarnClientSchedulerBackend( val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort - conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) + sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIHostPort) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( From ce59725b8703d18988e495dbaaf86ddde4bdfc5a Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 11 Sep 2014 17:28:36 -0700 Subject: [PATCH 41/76] [SPARK-3429] Don't include the empty string "" as a defaultAclUser Changes logging from ``` 14/09/05 02:01:08 INFO SecurityManager: Changing view acls to: aash, 14/09/05 02:01:08 INFO SecurityManager: Changing modify acls to: aash, 14/09/05 02:01:08 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(aash, ); users with modify permissions: Set(aash, ) ``` to ``` 14/09/05 02:28:28 INFO SecurityManager: Changing view acls to: aash 14/09/05 02:28:28 INFO SecurityManager: Changing modify acls to: aash 14/09/05 02:28:28 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(aash); users with modify permissions: Set(aash) ``` Note that the first set of logs have a Set of size 2 containing "aash" and the empty string "" cc tgravescs Author: Andrew Ash Closes #2286 from ash211/empty-default-acl and squashes the following commits: 18cc612 [Andrew Ash] Use .isEmpty instead of =="" cf973a1 [Andrew Ash] Don't include the empty string "" as a defaultAclUser --- core/src/main/scala/org/apache/spark/SecurityManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 12b15fe0815be..3832a780ec4bc 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -162,7 +162,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { // always add the current user and SPARK_USER to the viewAcls private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), - Option(System.getenv("SPARK_USER")).getOrElse("")) + Option(System.getenv("SPARK_USER")).getOrElse("")).filter(!_.isEmpty) setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) From f858f466862541c3faad76a1fa2391f1c17ec9dd Mon Sep 17 00:00:00 2001 From: Cody Koeninger Date: Thu, 11 Sep 2014 17:49:36 -0700 Subject: [PATCH 42/76] SPARK-3462 push down filters and projections into Unions Author: Cody Koeninger Closes #2345 from koeninger/SPARK-3462 and squashes the following commits: 5c8d24d [Cody Koeninger] SPARK-3462 remove now-unused parameter 0788691 [Cody Koeninger] SPARK-3462 add tests, handle compatible schema with different aliases, per marmbrus feedback ef47b3b [Cody Koeninger] SPARK-3462 push down filters and projections into Unions --- .../sql/catalyst/optimizer/Optimizer.scala | 48 ++++++++++++++ .../optimizer/UnionPushdownSuite.scala | 62 +++++++++++++++++++ 2 files changed, 110 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ddd4b3755d629..a4133feae8166 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -40,12 +40,60 @@ object Optimizer extends RuleExecutor[LogicalPlan] { SimplifyCasts, SimplifyCaseConversionExpressions) :: Batch("Filter Pushdown", FixedPoint(100), + UnionPushdown, CombineFilters, PushPredicateThroughProject, PushPredicateThroughJoin, ColumnPruning) :: Nil } +/** + * Pushes operations to either side of a Union. + */ +object UnionPushdown extends Rule[LogicalPlan] { + + /** + * Maps Attributes from the left side to the corresponding Attribute on the right side. + */ + def buildRewrites(union: Union): AttributeMap[Attribute] = { + assert(union.left.output.size == union.right.output.size) + + AttributeMap(union.left.output.zip(union.right.output)) + } + + /** + * Rewrites an expression so that it can be pushed to the right side of a Union operator. + * This method relies on the fact that the output attributes of a union are always equal + * to the left child's output. + */ + def pushToRight[A <: Expression](e: A, rewrites: AttributeMap[Attribute]): A = { + val result = e transform { + case a: Attribute => rewrites(a) + } + + // We must promise the compiler that we did not discard the names in the case of project + // expressions. This is safe since the only transformation is from Attribute => Attribute. + result.asInstanceOf[A] + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Push down filter into union + case Filter(condition, u @ Union(left, right)) => + val rewrites = buildRewrites(u) + Union( + Filter(condition, left), + Filter(pushToRight(condition, rewrites), right)) + + // Push down projection into union + case Project(projectList, u @ Union(left, right)) => + val rewrites = buildRewrites(u) + Union( + Project(projectList, left), + Project(projectList.map(pushToRight(_, rewrites)), right)) + } +} + + /** * Attempts to eliminate the reading of unneeded columns from the query plan using the following * transformations: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala new file mode 100644 index 0000000000000..dfef87bd9133d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ + +class UnionPushdownSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateAnalysisOperators) :: + Batch("Union Pushdown", Once, + UnionPushdown) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) + val testUnion = Union(testRelation, testRelation2) + + test("union: filter to each side") { + val query = testUnion.where('a === 1) + + val optimized = Optimize(query.analyze) + + val correctAnswer = + Union(testRelation.where('a === 1), testRelation2.where('d === 1)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("union: project to each side") { + val query = testUnion.select('b) + + val optimized = Optimize(query.analyze) + + val correctAnswer = + Union(testRelation.select('b), testRelation2.select('e)).analyze + + comparePlans(optimized, correctAnswer) + } +} From 33c7a738ae9f2d12425afad6f08a4fe0b7a5c6ab Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 11 Sep 2014 18:44:35 -0700 Subject: [PATCH 43/76] SPARK-2482: Resolve sbt warnings during build At the same time, import the `scala.language.postfixOps` and ` org.scalatest.time.SpanSugar._` cause `scala.language.postfixOps` doesn't work Author: witgo Closes #1330 from witgo/sbt_warnings3 and squashes the following commits: 179ba61 [witgo] Resolve sbt warnings during build --- .../scala/org/apache/spark/bagel/BagelSuite.scala | 2 -- .../scala/org/apache/spark/ContextCleanerSuite.scala | 1 - .../test/scala/org/apache/spark/DriverSuite.scala | 2 -- .../org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 1 - .../src/test/scala/org/apache/spark/ui/UISuite.scala | 1 - pom.xml | 1 - .../scala/org/apache/spark/repl/SparkILoop.scala | 12 ++++++------ .../spark/streaming/NetworkReceiverSuite.scala | 1 - .../spark/streaming/StreamingContextSuite.scala | 2 -- .../spark/streaming/StreamingListenerSuite.scala | 1 - .../scala/org/apache/spark/streaming/UISuite.scala | 1 - 11 files changed, 6 insertions(+), 19 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 55241d33cd3f0..ccb262a4ee02a 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -24,8 +24,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.storage.StorageLevel -import scala.language.postfixOps - class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 2744894277ae8..2e3fc5ef0e336 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -21,7 +21,6 @@ import java.lang.ref.WeakReference import scala.collection.mutable.{HashSet, SynchronizedSet} import scala.language.existentials -import scala.language.postfixOps import scala.util.Random import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 4b1d280624c57..5265ba904032f 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -26,8 +26,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils -import scala.language.postfixOps - class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 28197657e9bad..3b833f2e41867 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -22,7 +22,6 @@ import java.util.concurrent.Semaphore import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global -import scala.language.postfixOps import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2f56642956701..48790b59e7fbd 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -21,7 +21,6 @@ import java.net.ServerSocket import javax.servlet.http.HttpServletRequest import scala.io.Source -import scala.language.postfixOps import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server diff --git a/pom.xml b/pom.xml index e5f863e85445c..28763476f8313 100644 --- a/pom.xml +++ b/pom.xml @@ -839,7 +839,6 @@ -unchecked -deprecation -feature - -language:postfixOps -Xms1024m diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index d9eeffa86016a..e56b74edba88c 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -15,15 +15,15 @@ import scala.tools.nsc._ import scala.tools.nsc.backend.JavaPlatform import scala.tools.nsc.interpreter._ -import scala.tools.nsc.interpreter.{ Results => IR } -import Predef.{ println => _, _ } -import java.io.{ BufferedReader, FileReader } +import scala.tools.nsc.interpreter.{Results => IR} +import Predef.{println => _, _} +import java.io.{BufferedReader, FileReader} import java.net.URI import java.util.concurrent.locks.ReentrantLock import scala.sys.process.Process import scala.tools.nsc.interpreter.session._ -import scala.util.Properties.{ jdkHome, javaVersion } -import scala.tools.util.{ Javap } +import scala.util.Properties.{jdkHome, javaVersion} +import scala.tools.util.{Javap} import scala.annotation.tailrec import scala.collection.mutable.ListBuffer import scala.concurrent.ops @@ -33,7 +33,7 @@ import scala.tools.nsc.io.{File, Directory} import scala.reflect.NameTransformer._ import scala.tools.nsc.util.ScalaClassLoader._ import scala.tools.util._ -import scala.language.{implicitConversions, existentials} +import scala.language.{implicitConversions, existentials, postfixOps} import scala.reflect.{ClassTag, classTag} import scala.tools.reflect.StdRuntimeTags._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index f4e11f975de94..99c8d13231aac 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.language.postfixOps import org.apache.spark.SparkConf import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 7b33d3b235466..a3cabd6be02fe 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -29,8 +29,6 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ -import scala.language.postfixOps - class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 2861f5335ae36..84fed95a75e67 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global -import scala.language.postfixOps import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 4c7e43c2943c9..8e30118266855 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming import scala.io.Source -import scala.language.postfixOps import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ From 42904b8d013e71d03e301c3da62e33b4cc2eb54e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 11 Sep 2014 18:53:26 -0700 Subject: [PATCH 44/76] [SPARK-3465] fix task metrics aggregation in local mode Before overwrite t.taskMetrics, take a deepcopy of it. Author: Davies Liu Closes #2338 from davies/fix_metric and squashes the following commits: a5cdb63 [Davies Liu] Merge branch 'master' into fix_metric 7c879e0 [Davies Liu] add more comments 754b5b8 [Davies Liu] copy taskMetrics only when isLocal is true 5ca26dc [Davies Liu] fix task metrics aggregation in local mode --- .../scala/org/apache/spark/executor/Executor.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index dd903dc65d204..acae448a9c66f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -360,7 +360,16 @@ private[spark] class Executor( if (!taskRunner.attemptedTask.isEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => metrics.updateShuffleReadMetrics - tasksMetrics += ((taskRunner.taskId, metrics)) + if (isLocal) { + // JobProgressListener will hold an reference of it during + // onExecutorMetricsUpdate(), then JobProgressListener can not see + // the changes of metrics any more, so make a deep copy of it + val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) + tasksMetrics += ((taskRunner.taskId, copiedMetrics)) + } else { + // It will be copied by serialization + tasksMetrics += ((taskRunner.taskId, metrics)) + } } } } From b8634df1f1eb6ce909bec779522c9c9912c7d06a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 12 Sep 2014 01:37:59 -0700 Subject: [PATCH 45/76] [SPARK-3160] [SPARK-3494] [mllib] DecisionTree: eliminate pre-allocated nodes, parentImpurities arrays. Memory calc bug fix. This PR includes some code simplifications and re-organization which will be helpful for implementing random forests. The main changes are that the nodes and parentImpurities arrays are no longer pre-allocated in the main train() method. Also added 2 bug fixes: * maxMemoryUsage calculation * over-allocation of space for bins in DTStatsAggregator for unordered features. Relation to RFs: * Since RFs will be deeper and will therefore be more likely sparse (not full trees), it could be a cost savings to avoid pre-allocating a full tree. * The associated re-organization also reduces bookkeeping, which will make RFs easier to implement. * The return code doneTraining may be generalized to include cases such as nodes ready for local training. Details: No longer pre-allocate parentImpurities array in main train() method. * parentImpurities values are now stored in individual nodes (in Node.stats.impurity). * These were not really needed. They were used in calculateGainForSplit(), but they can be calculated anyways using parentNodeAgg. No longer using Node.build since tree structure is constructed on-the-fly. * Did not eliminate since it is public (Developer) API. Marked as deprecated. Eliminated pre-allocated nodes array in main train() method. * Nodes are constructed and added to the tree structure as needed during training. * Moved tree construction from main train() method into findBestSplitsPerGroup() since there is no need to keep the (split, gain) array for an entire level of nodes. Only one element of that array is needed at a time, so we do not the array. findBestSplits() now returns 2 items: * rootNode (newly created root node on first iteration, same root node on later iterations) * doneTraining (indicating if all nodes at that level were leafs) Updated DecisionTreeSuite. Notes: * Improved test "Second level node building with vs. without groups" ** generateOrderedLabeledPoints() modified so that it really does require 2 levels of internal nodes. * Related update: Added Node.deepCopy (private[tree]), used for test suite CC: mengxr Author: Joseph K. Bradley Closes #2341 from jkbradley/dt-spark-3160 and squashes the following commits: 07dd1ee [Joseph K. Bradley] Fixed overflow bug with computing maxMemoryUsage in DecisionTree. Also fixed bug with over-allocating space in DTStatsAggregator for unordered features. debe072 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 5c4ac33 [Joseph K. Bradley] Added check in Strategy to make sure minInstancesPerNode >= 1 0dd4d87 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 306120f [Joseph K. Bradley] Fixed typo in DecisionTreeModel.scala doc eaa1dcf [Joseph K. Bradley] Added topNode doc in DecisionTree and scalastyle fix d4d7864 [Joseph K. Bradley] Marked Node.build as deprecated d4dbb99 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 1a8f0ad [Joseph K. Bradley] Eliminated pre-allocated nodes array in main train() method. * Nodes are constructed and added to the tree structure as needed during training. 2ab763b [Joseph K. Bradley] Simplifications to DecisionTree code: --- .../spark/mllib/tree/DecisionTree.scala | 191 +++++------- .../mllib/tree/configuration/Strategy.scala | 3 + .../mllib/tree/impl/DTStatsAggregator.scala | 11 +- .../tree/impl/DecisionTreeMetadata.scala | 3 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../apache/spark/mllib/tree/model/Node.scala | 37 +++ .../spark/mllib/tree/DecisionTreeSuite.scala | 277 +++++++++--------- 7 files changed, 268 insertions(+), 256 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 98596569b8c95..56bb8812100a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -87,17 +87,11 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val maxDepth = strategy.maxDepth require(maxDepth <= 30, s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") - // Number of nodes to allocate: max number of nodes possible given the depth of the tree, plus 1 - val maxNumNodesPlus1 = Node.startIndexInLevel(maxDepth + 1) - // Initialize an array to hold parent impurity calculations for each node. - val parentImpurities = new Array[Double](maxNumNodesPlus1) - // dummy value for top node (updated during first split calculation) - val nodes = new Array[Node](maxNumNodesPlus1) // Calculate level for single group construction // Max memory usage for aggregates - val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 + val maxMemoryUsage = strategy.maxMemoryInMB * 1024L * 1024L logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") // TODO: Calculate memory usage more precisely. val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) @@ -120,81 +114,35 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * beforehand and is not used in later levels. */ + var topNode: Node = null // set on first iteration var level = 0 var break = false while (level <= maxDepth && !break) { - logDebug("#####################################") logDebug("level = " + level) logDebug("#####################################") // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel: Array[(Split, InformationGainStats, Predict)] = - DecisionTree.findBestSplits(treeInput, parentImpurities, - metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) + val (tmpTopNode: Node, doneTraining: Boolean) = DecisionTree.findBestSplits(treeInput, + metadata, level, topNode, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") - val levelNodeIndexOffset = Node.startIndexInLevel(level) - for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { - val nodeIndex = levelNodeIndexOffset + index - - // Extract info for this node (index) at the current level. - timer.start("extractNodeInfo") - val split = nodeSplitStats._1 - val stats = nodeSplitStats._2 - val predict = nodeSplitStats._3.predict - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - nodes(nodeIndex) = node - timer.stop("extractNodeInfo") - - if (level != 0) { - // Set parent. - val parentNodeIndex = Node.parentIndex(nodeIndex) - if (Node.isLeftChild(nodeIndex)) { - nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) - } else { - nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) - } - } - // Extract info for nodes at the next lower level. - timer.start("extractInfoForLowerLevels") - if (level < maxDepth) { - val leftChildIndex = Node.leftChildIndex(nodeIndex) - val leftImpurity = stats.leftImpurity - logDebug("leftChildIndex = " + leftChildIndex + ", impurity = " + leftImpurity) - parentImpurities(leftChildIndex) = leftImpurity - - val rightChildIndex = Node.rightChildIndex(nodeIndex) - val rightImpurity = stats.rightImpurity - logDebug("rightChildIndex = " + rightChildIndex + ", impurity = " + rightImpurity) - parentImpurities(rightChildIndex) = rightImpurity - } - timer.stop("extractInfoForLowerLevels") - logDebug("final best split = " + split) + if (level == 0) { + topNode = tmpTopNode } - require(Node.maxNodesInLevel(level) == splitsStatsForLevel.length) - // Check whether all the nodes at the current level at leaves. - val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) - logDebug("all leaf = " + allLeaf) - if (allLeaf) { - break = true // no more tree construction - } else { - level += 1 + if (doneTraining) { + break = true + logDebug("done training") } + + level += 1 } logDebug("#####################################") logDebug("Extracting tree model") logDebug("#####################################") - // Initialize the top or root node of the tree. - val topNode = nodes(1) - // Build the full tree using the node info calculated in the level-wise best split calculations. - topNode.build(nodes) - timer.stop("total") logInfo("Internal timing for DecisionTree:") @@ -409,24 +357,26 @@ object DecisionTree extends Serializable with Logging { * multiple groups if the level-wise training task could lead to memory overflow. * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree + * @param topNode Root node of the tree (or invalid node when training first level). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. - * @return array (over nodes) of splits with best split for each node at a given level. + * @return (root, doneTraining) where: + * root = Root node (which is newly created on the first iteration), + * doneTraining = true if no more internal nodes were created. */ private[tree] def findBestSplits( input: RDD[TreePoint], - parentImpurities: Array[Double], metadata: DecisionTreeMetadata, level: Int, - nodes: Array[Node], + topNode: Node, splits: Array[Array[Split]], bins: Array[Array[Bin]], maxLevelForSingleGroup: Int, - timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats, Predict)] = { + timer: TimeTracker = new TimeTracker): (Node, Boolean) = { + // split into groups to avoid memory overflow during aggregation if (level > maxLevelForSingleGroup) { // When information for all nodes at a given level cannot be stored in memory, @@ -435,18 +385,18 @@ object DecisionTree extends Serializable with Logging { // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) - var bestSplits = new Array[(Split, InformationGainStats, Predict)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 + var doneTraining = true while (groupIndex < numGroups) { - val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, metadata, level, - nodes, splits, bins, timer, numGroups, groupIndex) - bestSplits = Array.concat(bestSplits, bestSplitsForGroup) + val (tmpRoot, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, + topNode, splits, bins, timer, numGroups, groupIndex) + doneTraining = doneTraining && doneTrainingGroup groupIndex += 1 } - bestSplits + (topNode, doneTraining) // Not first iteration, so topNode was already set. } else { - findBestSplitsPerGroup(input, parentImpurities, metadata, level, nodes, splits, bins, timer) + findBestSplitsPerGroup(input, metadata, level, topNode, splits, bins, timer) } } @@ -586,27 +536,27 @@ object DecisionTree extends Serializable with Logging { * Returns an array of optimal splits for a group of nodes at a given level * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param nodes Array of all nodes in the tree. Used for matching data points to nodes. + * @param topNode Root node of the tree (or invalid node when training first level). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. - * @return array of splits with best splits for all nodes at a given level. + * @return (root, doneTraining) where: + * root = Root node (which is newly created on the first iteration), + * doneTraining = true if no more internal nodes were created. */ private def findBestSplitsPerGroup( input: RDD[TreePoint], - parentImpurities: Array[Double], metadata: DecisionTreeMetadata, level: Int, - nodes: Array[Node], + topNode: Node, splits: Array[Array[Split]], bins: Array[Array[Bin]], timer: TimeTracker, numGroups: Int = 1, - groupIndex: Int = 0): Array[(Split, InformationGainStats, Predict)] = { + groupIndex: Int = 0): (Node, Boolean) = { /* * The high-level descriptions of the best split optimizations are noted here. @@ -663,7 +613,7 @@ object DecisionTree extends Serializable with Logging { 0 } else { val globalNodeIndex = - predictNodeIndex(nodes(1), treePoint.binnedFeatures, bins, metadata.unorderedFeatures) + predictNodeIndex(topNode, treePoint.binnedFeatures, bins, metadata.unorderedFeatures) globalNodeIndex - globalNodeIndexOffset } } @@ -706,33 +656,63 @@ object DecisionTree extends Serializable with Logging { // Calculate best splits for all nodes at a given level timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats, Predict)](numNodes) - // Iterating over all nodes at this level + // On the first iteration, we need to get and return the newly created root node. + var newTopNode: Node = topNode + + // Iterate over all nodes at this level var nodeIndex = 0 + var internalNodeCount = 0 while (nodeIndex < numNodes) { - val nodeImpurity = parentImpurities(globalNodeIndexOffset + nodeIndex) - logDebug("node impurity = " + nodeImpurity) - bestSplits(nodeIndex) = - binsToBestSplit(binAggregates, nodeIndex, nodeImpurity, level, metadata, splits) - logDebug("best split = " + bestSplits(nodeIndex)._1) + val (split: Split, stats: InformationGainStats, predict: Predict) = + binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits) + logDebug("best split = " + split) + + val globalNodeIndex = globalNodeIndexOffset + nodeIndex + + // Extract info for this node at the current level. + val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth) + val node = + new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats)) + logDebug("Node = " + node) + + if (!isLeaf) { + internalNodeCount += 1 + } + if (level == 0) { + newTopNode = node + } else { + // Set parent. + val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode) + if (Node.isLeftChild(globalNodeIndex)) { + parentNode.leftNode = Some(node) + } else { + parentNode.rightNode = Some(node) + } + } + if (level < metadata.maxDepth) { + logDebug("leftChildIndex = " + Node.leftChildIndex(globalNodeIndex) + + ", impurity = " + stats.leftImpurity) + logDebug("rightChildIndex = " + Node.rightChildIndex(globalNodeIndex) + + ", impurity = " + stats.rightImpurity) + } + nodeIndex += 1 } timer.stop("chooseSplits") - bestSplits + val doneTraining = internalNodeCount == 0 + (newTopNode, doneTraining) } /** * Calculate the information gain for a given (feature, split) based upon left/right aggregates. * @param leftImpurityCalculator left node aggregates for this (feature, split) * @param rightImpurityCalculator right node aggregate for this (feature, split) - * @param topImpurity impurity of the parent node * @return information gain and statistics for all splits */ private def calculateGainForSplit( leftImpurityCalculator: ImpurityCalculator, rightImpurityCalculator: ImpurityCalculator, - topImpurity: Double, level: Int, metadata: DecisionTreeMetadata): InformationGainStats = { val leftCount = leftImpurityCalculator.count @@ -747,14 +727,10 @@ object DecisionTree extends Serializable with Logging { val totalCount = leftCount + rightCount - // impurity of parent node - val impurity = if (level > 0) { - topImpurity - } else { - val parentNodeAgg = leftImpurityCalculator.copy - parentNodeAgg.add(rightImpurityCalculator) - parentNodeAgg.calculate() - } + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + + val impurity = parentNodeAgg.calculate() val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 val rightImpurity = rightImpurityCalculator.calculate() @@ -795,19 +771,15 @@ object DecisionTree extends Serializable with Logging { * Find the best split for a node. * @param binAggregates Bin statistics. * @param nodeIndex Index for node to split in this (level, group). - * @param nodeImpurity Impurity of the node (nodeIndex). * @return tuple for best split: (Split, information gain) */ private def binsToBestSplit( binAggregates: DTStatsAggregator, nodeIndex: Int, - nodeImpurity: Double, level: Int, metadata: DecisionTreeMetadata, splits: Array[Array[Split]]): (Split, InformationGainStats, Predict) = { - logDebug("node impurity = " + nodeImpurity) - // calculate predict only once var predict: Option[Predict] = None @@ -831,8 +803,7 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = - calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) @@ -845,8 +816,7 @@ object DecisionTree extends Serializable with Logging { val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = - calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) @@ -917,8 +887,7 @@ object DecisionTree extends Serializable with Logging { binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = - calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = @@ -937,8 +906,8 @@ object DecisionTree extends Serializable with Logging { /** * Get the number of values to be stored per node in the bin aggregates. */ - private def getElementsPerNode(metadata: DecisionTreeMetadata): Int = { - val totalBins = metadata.numBins.sum + private def getElementsPerNode(metadata: DecisionTreeMetadata): Long = { + val totalBins = metadata.numBins.map(_.toLong).sum if (metadata.isClassification) { metadata.numClasses * totalBins } else { 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 987fe632c91ed..31d1e8ac30eea 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 @@ -75,6 +75,9 @@ class Strategy ( if (algo == Classification) { require(numClassesForClassification >= 2) } + require(minInstancesPerNode >= 1, + s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") + val isMulticlassClassification = algo == Classification && numClassesForClassification > 2 val isMulticlassWithCategoricalFeatures diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index 866d85a79bea1..61a94246711bf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -65,14 +65,7 @@ private[tree] class DTStatsAggregator( * Offset for each feature for calculating indices into the [[allStats]] array. */ private val featureOffsets: Array[Int] = { - def featureOffsetsCalc(total: Int, featureIndex: Int): Int = { - if (isUnordered(featureIndex)) { - total + 2 * numBins(featureIndex) - } else { - total + numBins(featureIndex) - } - } - Range(0, numFeatures).scanLeft(0)(featureOffsetsCalc).map(statsSize * _).toArray + numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) } /** @@ -149,7 +142,7 @@ private[tree] class DTStatsAggregator( s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + s" but was called for ordered feature $featureIndex.") val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) - (baseOffset, baseOffset + numBins(featureIndex) * statsSize) + (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } /** 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 5ceaa8154d11a..b6d49e5555b1a 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 @@ -46,6 +46,7 @@ private[tree] class DecisionTreeMetadata( val numBins: Array[Int], val impurity: Impurity, val quantileStrategy: QuantileStrategy, + val maxDepth: Int, val minInstancesPerNode: Int, val minInfoGain: Double) extends Serializable { @@ -129,7 +130,7 @@ private[tree] object DecisionTreeMetadata { new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, - strategy.impurity, strategy.quantileCalculationStrategy, + strategy.impurity, strategy.quantileCalculationStrategy, strategy.maxDepth, strategy.minInstancesPerNode, strategy.minInfoGain) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 0594fd0749d21..271b2c4ad813e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -46,7 +46,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * Predict values for the given data set using the model trained. * * @param features RDD representing data points to be predicted - * @return RDD[Int] where each entry contains the corresponding prediction + * @return RDD of predictions for each of the given data points */ def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 5b8a4cbed2306..5f0095d23c7ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -55,6 +55,8 @@ class Node ( * build the left node and right nodes if not leaf * @param nodes array of nodes */ + @deprecated("build should no longer be used since trees are constructed on-the-fly in training", + "1.2.0") def build(nodes: Array[Node]): Unit = { logDebug("building node " + id + " at level " + Node.indexToLevel(id)) logDebug("id = " + id + ", split = " + split) @@ -93,6 +95,23 @@ class Node ( } } + /** + * Returns a deep copy of the subtree rooted at this node. + */ + private[tree] def deepCopy(): Node = { + val leftNodeCopy = if (leftNode.isEmpty) { + None + } else { + Some(leftNode.get.deepCopy()) + } + val rightNodeCopy = if (rightNode.isEmpty) { + None + } else { + Some(rightNode.get.deepCopy()) + } + new Node(id, predict, isLeaf, split, leftNodeCopy, rightNodeCopy, stats) + } + /** * Get the number of nodes in tree below this node, including leaf nodes. * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. @@ -190,4 +209,22 @@ private[tree] object Node { */ def startIndexInLevel(level: Int): Int = 1 << level + /** + * Traces down from a root node to get the node with the given node index. + * This assumes the node exists. + */ + def getNode(nodeIndex: Int, rootNode: Node): Node = { + var tmpNode: Node = rootNode + var levelsToGo = indexToLevel(nodeIndex) + while (levelsToGo > 0) { + if ((nodeIndex & (1 << levelsToGo - 1)) == 0) { + tmpNode = tmpNode.leftNode.get + } else { + tmpNode = tmpNode.rightNode.get + } + levelsToGo -= 1 + } + tmpNode + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index fd8547c1660fc..1bd7ea05c46c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -270,19 +270,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 0) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode: Node, doneTraining: Boolean) = + DecisionTree.findBestSplits(treeInput, metadata, 0, null, splits, bins, 10) - val split = bestSplits(0)._1 + val split = rootNode.split.get assert(split.categories === List(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) - val stats = bestSplits(0)._2 - val predict = bestSplits(0)._3 + val stats = rootNode.stats.get assert(stats.gain > 0) - assert(predict.predict === 1) - assert(predict.prob === 0.6) + assert(rootNode.predict === 1) assert(stats.impurity > 0.2) } @@ -303,19 +301,18 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - val split = bestSplits(0)._1 + val split = rootNode.split.get assert(split.categories.length === 1) assert(split.categories.contains(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) - val stats = bestSplits(0)._2 - val predict = bestSplits(0)._3.predict + val stats = rootNode.stats.get assert(stats.gain > 0) - assert(predict === 0.6) + assert(rootNode.predict === 0.6) assert(stats.impurity > 0.2) } @@ -356,13 +353,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) } test("Binary classification stump with fixed label 1 for Gini") { @@ -382,14 +382,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._3.predict === 1) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) + assert(rootNode.predict === 1) } test("Binary classification stump with fixed label 0 for Entropy") { @@ -409,14 +412,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._3.predict === 0) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) + assert(rootNode.predict === 0) } test("Binary classification stump with fixed label 1 for Entropy") { @@ -436,14 +442,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._3.predict === 1) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) + assert(rootNode.predict === 1) } test("Second level node building with vs. without groups") { @@ -459,40 +468,46 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) // Train a 1-node model - val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) + val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, + numClassesForClassification = 2, maxBins = 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val nodes: Array[Node] = new Array[Node](8) - nodes(1) = modelOneNode.topNode - nodes(1).leftNode = None - nodes(1).rightNode = None - - val parentImpurities = Array(0, 0.5, 0.5, 0.5) + val rootNodeCopy1 = modelOneNode.topNode.deepCopy() + val rootNodeCopy2 = modelOneNode.topNode.deepCopy() // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, nodes, - splits, bins, 10) - assert(bestSplits.length === 2) - assert(bestSplits(0)._2.gain > 0) - assert(bestSplits(1)._2.gain > 0) + val (rootNode, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, + rootNodeCopy1, splits, bins, 10) + assert(rootNode.leftNode.nonEmpty) + assert(rootNode.rightNode.nonEmpty) + val children1 = new Array[Node](2) + children1(0) = rootNode.leftNode.get + children1(1) = rootNode.rightNode.get // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second // level tree construction. - val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, - nodes, splits, bins, 0) - assert(bestSplitsWithGroups.length === 2) - assert(bestSplitsWithGroups(0)._2.gain > 0) - assert(bestSplitsWithGroups(1)._2.gain > 0) + val (rootNode2, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, + rootNodeCopy2, splits, bins, 0) + assert(rootNode2.leftNode.nonEmpty) + assert(rootNode2.rightNode.nonEmpty) + val children2 = new Array[Node](2) + children2(0) = rootNode2.leftNode.get + children2(1) = rootNode2.rightNode.get // Verify whether the splits obtained using single group and multiple group level // construction strategies are the same. - for (i <- 0 until bestSplits.length) { - assert(bestSplits(i)._1 === bestSplitsWithGroups(i)._1) - assert(bestSplits(i)._2.gain === bestSplitsWithGroups(i)._2.gain) - assert(bestSplits(i)._2.impurity === bestSplitsWithGroups(i)._2.impurity) - assert(bestSplits(i)._2.leftImpurity === bestSplitsWithGroups(i)._2.leftImpurity) - assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) - assert(bestSplits(i)._3.predict === bestSplitsWithGroups(i)._3.predict) + for (i <- 0 until 2) { + assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) + assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) + assert(children1(i).split === children2(i).split) + assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) + val stats1 = children1(i).stats.get + val stats2 = children2(i).stats.get + assert(stats1.gain === stats2.gain) + assert(stats1.impurity === stats2.impurity) + assert(stats1.leftImpurity === stats2.leftImpurity) + assert(stats1.rightImpurity === stats2.rightImpurity) + assert(children1(i).predict === children2(i).predict) } } @@ -508,15 +523,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - assert(bestSplit.feature === 0) - assert(bestSplit.categories.length === 1) - assert(bestSplit.categories.contains(1)) - assert(bestSplit.featureType === Categorical) + val split = rootNode.split.get + assert(split.feature === 0) + assert(split.categories.length === 1) + assert(split.categories.contains(1)) + assert(split.featureType === Categorical) } test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { @@ -573,16 +587,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) - - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - assert(bestSplit.feature === 0) - assert(bestSplit.categories.length === 1) - assert(bestSplit.categories.contains(1)) - assert(bestSplit.featureType === Categorical) - val gain = bestSplits(0)._2 + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + assert(split.categories.length === 1) + assert(split.categories.contains(1)) + assert(split.featureType === Categorical) + + val gain = rootNode.stats.get assert(gain.leftImpurity === 0) assert(gain.rightImpurity === 0) } @@ -600,16 +614,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) - - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplit.feature === 1) - assert(bestSplit.featureType === Continuous) - assert(bestSplit.threshold > 1980) - assert(bestSplit.threshold < 2020) + val split = rootNode.split.get + assert(split.feature === 1) + assert(split.featureType === Continuous) + assert(split.threshold > 1980) + assert(split.threshold < 2020) } @@ -627,16 +639,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - - assert(bestSplit.feature === 1) - assert(bestSplit.featureType === Continuous) - assert(bestSplit.threshold > 1980) - assert(bestSplit.threshold < 2020) + val split = rootNode.split.get + assert(split.feature === 1) + assert(split.featureType === Continuous) + assert(split.threshold > 1980) + assert(split.threshold < 2020) } test("Multiclass classification stump with 10-ary (ordered) categorical features") { @@ -652,15 +662,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - assert(bestSplit.feature === 0) - assert(bestSplit.categories.length === 1) - assert(bestSplit.categories.contains(1.0)) - assert(bestSplit.featureType === Categorical) + val split = rootNode.split.get + assert(split.feature === 0) + assert(split.categories.length === 1) + assert(split.categories.contains(1.0)) + assert(split.featureType === Categorical) } test("Multiclass classification tree with 10-ary (ordered) categorical features," + @@ -698,12 +707,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length == 1) - val bestInfoStats = bestSplits(0)._2 - assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + val gain = rootNode.stats.get + assert(gain == InformationGainStats.invalidInformationGainStats) } test("don't choose split that doesn't satisfy min instance per node requirements") { @@ -722,14 +730,13 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length == 1) - val bestSplit = bestSplits(0)._1 - val bestSplitStats = bestSplits(0)._1 - assert(bestSplit.feature == 1) - assert(bestSplitStats != InformationGainStats.invalidInformationGainStats) + val split = rootNode.split.get + val gain = rootNode.stats.get + assert(split.feature == 1) + assert(gain != InformationGainStats.invalidInformationGainStats) } test("split must satisfy min info gain requirements") { @@ -754,12 +761,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length == 1) - val bestInfoStats = bestSplits(0)._2 - assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + val gain = rootNode.stats.get + assert(gain == InformationGainStats.invalidInformationGainStats) } } @@ -786,13 +792,16 @@ object DecisionTreeSuite { def generateOrderedLabeledPoints(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000) { - if (i < 600) { - val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i)) - arr(i) = lp + val label = if (i < 100) { + 0.0 + } else if (i < 500) { + 1.0 + } else if (i < 900) { + 0.0 } else { - val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 1000.0 - i)) - arr(i) = lp + 1.0 } + arr(i) = new LabeledPoint(label, Vectors.dense(i.toDouble, 1000.0 - i)) } arr } From f116f76bf1f1610905ca094c8edc53151a78d2f4 Mon Sep 17 00:00:00 2001 From: "Mark G. Whitney" Date: Fri, 12 Sep 2014 08:08:58 -0500 Subject: [PATCH 46/76] [SPARK-2558][DOCS] Add --queue example to YARN doc Put original YARN queue spark-submit arg description in running-on-yarn html table and example command line Author: Mark G. Whitney Closes #2218 from kramimus/2258-yarndoc and squashes the following commits: 4b5d808 [Mark G. Whitney] remove yarn queue config f8cda0d [Mark G. Whitney] [SPARK-2558][DOCS] Add spark.yarn.queue description to YARN doc --- docs/running-on-yarn.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d8b22f3663d08..212248bcce1c1 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -155,6 +155,7 @@ For example: --driver-memory 4g \ --executor-memory 2g \ --executor-cores 1 \ + --queue thequeue \ lib/spark-examples*.jar \ 10 From 533377621f1e178e18fa0b79d653a11b66e4e250 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Fri, 12 Sep 2014 09:46:21 -0700 Subject: [PATCH 47/76] [PySpark] Add blank line so that Python RDD.top() docstring renders correctly Author: RJ Nowling Closes #2370 from rnowling/python_rdd_docstrings and squashes the following commits: 5230574 [RJ Nowling] Add blank line so that Python RDD.top() docstring renders correctly --- python/pyspark/rdd.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 5667154cb84a8..6ad5ab2a2d1ae 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1060,6 +1060,7 @@ def top(self, num, key=None): Get the top N elements from a RDD. Note: It returns the list sorted in descending order. + >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) [12] >>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2) From 8194fc662c08eb445444c207264e22361def54ea Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 12 Sep 2014 11:29:30 -0700 Subject: [PATCH 48/76] [SPARK-3481] [SQL] Eliminate the error log in local Hive comparison test Logically, we should remove the Hive Table/Database first and then reset the Hive configuration, repoint to the new data warehouse directory etc. Otherwise it raised exceptions like "Database doesn't not exists: default" in the local testing. Author: Cheng Hao Closes #2352 from chenghao-intel/test_hive and squashes the following commits: 74fd76b [Cheng Hao] eliminate the error log --- .../org/apache/spark/sql/hive/TestHive.scala | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 6974f3e581b97..a3bfd3a8f1fd2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -376,15 +376,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } - // It is important that we RESET first as broken hooks that might have been set could break - // other sql exec here. - runSqlHive("RESET") - // For some reason, RESET does not reset the following variables... - runSqlHive("set datanucleus.cache.collections=true") - runSqlHive("set datanucleus.cache.collections.lazy=true") - // Lots of tests fail if we do not change the partition whitelist from the default. - runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - loadedTables.clear() catalog.client.getAllTables("default").foreach { t => logDebug(s"Deleting table $t") @@ -410,6 +401,14 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { FunctionRegistry.unregisterTemporaryUDF(udfName) } + // It is important that we RESET first as broken hooks that might have been set could break + // other sql exec here. + runSqlHive("RESET") + // For some reason, RESET does not reset the following variables... + runSqlHive("set datanucleus.cache.collections=true") + runSqlHive("set datanucleus.cache.collections.lazy=true") + // Lots of tests fail if we do not change the partition whitelist from the default. + runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") configure() runSqlHive("USE default") From eae81b0bfdf3159be90f507a03853800aec1874a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Sep 2014 13:43:29 -0700 Subject: [PATCH 49/76] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #930 (close requested by 'andrewor14') Closes #867 (close requested by 'marmbrus') Closes #1829 (close requested by 'marmbrus') Closes #1131 (close requested by 'JoshRosen') Closes #1571 (close requested by 'andrewor14') Closes #2359 (close requested by 'andrewor14') From 15a564598fe63003652b1e24527c432080b5976c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 12 Sep 2014 14:08:38 -0700 Subject: [PATCH 50/76] [SPARK-3427] [GraphX] Avoid active vertex tracking in static PageRank GraphX's current implementation of static (fixed iteration count) PageRank uses the Pregel API. This unnecessarily tracks active vertices, even though in static PageRank all vertices are always active. Active vertex tracking incurs the following costs: 1. A shuffle per iteration to ship the active sets to the edge partitions. 2. A hash table creation per iteration at each partition to index the active sets for lookup. 3. A hash lookup per edge to check whether the source vertex is active. I reimplemented static PageRank using the lower-level GraphX API instead of the Pregel API. In benchmarks on a 16-node m2.4xlarge cluster, this provided a 23% speedup (from 514 s to 397 s, mean over 3 trials) for 10 iterations of PageRank on a synthetic graph with 10M vertices and 1.27B edges. Author: Ankur Dave Closes #2308 from ankurdave/SPARK-3427 and squashes the following commits: 449996a [Ankur Dave] Avoid unnecessary active vertex tracking in static PageRank --- .../apache/spark/graphx/lib/PageRank.scala | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 614555a054dfb..257e2f3a36115 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -79,30 +79,43 @@ object PageRank extends Logging { def run[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { - // Initialize the pagerankGraph with each edge attribute having + // Initialize the PageRank graph with each edge attribute having // weight 1/outDegree and each vertex with attribute 1.0. - val pagerankGraph: Graph[Double, Double] = graph + var rankGraph: Graph[Double, Double] = graph // Associate the degree with each vertex .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to the initial pagerank values - .mapVertices( (id, attr) => 1.0 ) - .cache() + .mapVertices( (id, attr) => resetProb ) - // Define the three functions needed to implement PageRank in the GraphX - // version of Pregel - def vertexProgram(id: VertexId, attr: Double, msgSum: Double): Double = - resetProb + (1.0 - resetProb) * msgSum - def sendMessage(edge: EdgeTriplet[Double, Double]) = - Iterator((edge.dstId, edge.srcAttr * edge.attr)) - def messageCombiner(a: Double, b: Double): Double = a + b - // The initial message received by all vertices in PageRank - val initialMessage = 0.0 + var iteration = 0 + var prevRankGraph: Graph[Double, Double] = null + while (iteration < numIter) { + rankGraph.cache() - // Execute pregel for a fixed number of iterations. - Pregel(pagerankGraph, initialMessage, numIter, activeDirection = EdgeDirection.Out)( - vertexProgram, sendMessage, messageCombiner) + // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and + // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. + val rankUpdates = rankGraph.mapReduceTriplets[Double]( + e => Iterator((e.dstId, e.srcAttr * e.attr)), _ + _) + + // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices + // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the + // edge partitions. + prevRankGraph = rankGraph + rankGraph = rankGraph.joinVertices(rankUpdates) { + (id, oldRank, msgSum) => resetProb + (1.0 - resetProb) * msgSum + }.cache() + + rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices + logInfo(s"PageRank finished iteration $iteration.") + prevRankGraph.vertices.unpersist(false) + prevRankGraph.edges.unpersist(false) + + iteration += 1 + } + + rankGraph } /** From 1d767967e925f1d727957c2d43383ef6ad2c5d5e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 12 Sep 2014 16:48:28 -0500 Subject: [PATCH 51/76] SPARK-3014. Log a more informative messages in a couple failure scenario... ...s Author: Sandy Ryza Closes #1934 from sryza/sandy-spark-3014 and squashes the following commits: ae19cc1 [Sandy Ryza] SPARK-3014. Log a more informative messages in a couple failure scenarios --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 6 ++++-- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 ++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0fdb5ae3c2e40..5ed3575816a38 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} -import java.lang.reflect.InvocationTargetException +import java.lang.reflect.{Modifier, InvocationTargetException} import java.net.URL import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -323,7 +323,9 @@ object SparkSubmit { } val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) - + if (!Modifier.isStatic(mainMethod.getModifiers)) { + throw new IllegalStateException("The main method in the given main class must be static") + } try { mainMethod.invoke(null, childArgs.toArray) } catch { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 878b6db546032..735d7723b0ce6 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -283,11 +283,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } val sparkContext = sparkContextRef.get() - assert(sparkContext != null || count >= numTries) if (sparkContext == null) { - logError( - "Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".format( - count * waitTime, numTries)) + logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" + + " log output for errors. Failing the application.").format(numTries * waitTime)) } sparkContext } From af2583826c15d2a4e2732017ea20feeff0fb79f6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 12 Sep 2014 14:54:42 -0700 Subject: [PATCH 52/76] [SPARK-3217] Add Guava to classpath when SPARK_PREPEND_CLASSES is set. When that option is used, the compiled classes from the build directory are prepended to the classpath. Now that we avoid packaging Guava, that means we have classes referencing the original Guava location in the app's classpath, so errors happen. For that case, add Guava manually to the classpath. Note: if Spark is compiled with "-Phadoop-provided", it's tricky to make things work with SPARK_PREPEND_CLASSES, because you need to add the Hadoop classpath using SPARK_CLASSPATH and that means the older Hadoop Guava overrides the newer one Spark needs. So someone using SPARK_PREPEND_CLASSES needs to remember to not use that profile. Author: Marcelo Vanzin Closes #2141 from vanzin/SPARK-3217 and squashes the following commits: b967324 [Marcelo Vanzin] [SPARK-3217] Add Guava to classpath when SPARK_PREPEND_CLASSES is set. --- bin/compute-classpath.sh | 1 + core/pom.xml | 27 +++++++++++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 15c6779402994..0f63e36d8aeca 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -43,6 +43,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ "classes ahead of assembly." >&2 CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" diff --git a/core/pom.xml b/core/pom.xml index b2b788a4bc13b..2a81f6df289c0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -351,6 +351,33 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory} + false + false + true + true + guava + true + + + + From 25311c2c545a60eb9dcf704814d4600987852155 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 12 Sep 2014 20:31:11 -0500 Subject: [PATCH 53/76] [SPARK-3456] YarnAllocator on alpha can lose container requests to RM Author: Thomas Graves Closes #2373 from tgravescs/SPARK-3456 and squashes the following commits: 77e9532 [Thomas Graves] [SPARK-3456] YarnAllocator on alpha can lose container requests to RM --- .../spark/deploy/yarn/YarnAllocationHandler.scala | 11 ++++++----- .../org/apache/spark/deploy/yarn/YarnAllocator.scala | 8 ++++++-- .../spark/deploy/yarn/YarnAllocationHandler.scala | 3 ++- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 5a1b42c1e17d5..6c93d8582330b 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -48,16 +48,17 @@ private[yarn] class YarnAllocationHandler( private val lastResponseId = new AtomicInteger() private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() - override protected def allocateContainers(count: Int): YarnAllocateResponse = { + override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { var resourceRequests: List[ResourceRequest] = null - logDebug("numExecutors: " + count) + logDebug("asking for additional executors: " + count + " with already pending: " + pending) + val totalNumAsk = count + pending if (count <= 0) { resourceRequests = List() } else if (preferredHostToCount.isEmpty) { logDebug("host preferences is empty") resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) + AllocationType.ANY, null, totalNumAsk, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -80,7 +81,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests: ResourceRequest = createResourceRequest( AllocationType.ANY, resource = null, - count, + totalNumAsk, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( @@ -103,7 +104,7 @@ private[yarn] class YarnAllocationHandler( req.addAllReleases(releasedContainerList) if (count > 0) { - logInfo("Allocating %d executor containers with %d of memory each.".format(count, + logInfo("Allocating %d executor containers with %d of memory each.".format(totalNumAsk, executorMemory + memoryOverhead)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 0b8744f4b8bdf..299e38a5eb9c0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -112,6 +112,9 @@ private[yarn] abstract class YarnAllocator( def allocateResources() = { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() + // this is needed by alpha, do it here since we add numPending right after this + val executorsPending = numPendingAllocate.get() + if (missing > 0) { numPendingAllocate.addAndGet(missing) logInfo("Will Allocate %d executor containers, each with %d memory".format( @@ -121,7 +124,7 @@ private[yarn] abstract class YarnAllocator( logDebug("Empty allocation request ...") } - val allocateResponse = allocateContainers(missing) + val allocateResponse = allocateContainers(missing, executorsPending) val allocatedContainers = allocateResponse.getAllocatedContainers() if (allocatedContainers.size > 0) { @@ -435,9 +438,10 @@ private[yarn] abstract class YarnAllocator( * * @param count Number of containers to allocate. * If zero, should still contact RM (as a heartbeat). + * @param pending Number of containers pending allocate. Only used on alpha. * @return Response to the allocation request. */ - protected def allocateContainers(count: Int): YarnAllocateResponse + protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse /** Called to release a previously allocated container. */ protected def releaseContainer(container: Container): Unit diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 5438f151ac0ad..e44a8db41b97e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -47,7 +47,8 @@ private[yarn] class YarnAllocationHandler( amClient.releaseAssignedContainer(container.getId()) } - override protected def allocateContainers(count: Int): YarnAllocateResponse = { + // pending isn't used on stable as the AMRMClient handles incremental asks + override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { addResourceRequests(count) // We have already set the container request. Poll the ResourceManager for a response. From 71af030b46a89aaa9a87f18f56b9e1f1cd8ce2e7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 12 Sep 2014 18:42:50 -0700 Subject: [PATCH 54/76] [SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy --- python/pyspark/cloudpickle.py | 168 ++++++++++++++-------------------- python/pyspark/daemon.py | 6 +- python/pyspark/serializers.py | 10 +- python/pyspark/tests.py | 85 +++++++++++++++-- python/run-tests | 21 +++++ 5 files changed, 172 insertions(+), 118 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 80e51d1a583a0..32dda3888c62d 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -52,35 +52,19 @@ import itertools from copy_reg import _extension_registry, _inverted_registry, _extension_cache import new -import dis import traceback +import platform -#relevant opcodes -STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) -DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) -LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) -GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +PyImp = platform.python_implementation() -HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) -EXTENDED_ARG = chr(dis.EXTENDED_ARG) import logging cloudLog = logging.getLogger("Cloud.Transport") -try: - import ctypes -except (MemoryError, ImportError): - logging.warning('Exception raised on importing ctypes. Likely python bug.. some functionality will be disabled', exc_info = True) - ctypes = None - PyObject_HEAD = None -else: - - # for reading internal structures - PyObject_HEAD = [ - ('ob_refcnt', ctypes.c_size_t), - ('ob_type', ctypes.c_void_p), - ] +if PyImp == "PyPy": + # register builtin type in `new` + new.method = types.MethodType try: from cStringIO import StringIO @@ -225,6 +209,8 @@ def save_function(self, obj, name=None, pack=struct.pack): if themodule: self.modules.add(themodule) + if getattr(themodule, name, None) is obj: + return self.save_global(obj, name) if not self.savedDjangoEnv: #hack for django - if we detect the settings module, we transport it @@ -306,44 +292,28 @@ def save_function_tuple(self, func, forced_imports): # create a skeleton function object and memoize it save(_make_skel_func) - save((code, len(closure), base_globals)) + save((code, closure, base_globals)) write(pickle.REDUCE) self.memoize(func) # save the rest of the func data needed by _fill_function save(f_globals) save(defaults) - save(closure) save(dct) write(pickle.TUPLE) write(pickle.REDUCE) # applies _fill_function on the tuple @staticmethod - def extract_code_globals(co): + def extract_code_globals(code): """ Find all globals names read or written to by codeblock co """ - code = co.co_code - names = co.co_names - out_names = set() - - n = len(code) - i = 0 - extended_arg = 0 - while i < n: - op = code[i] - - i = i+1 - if op >= HAVE_ARGUMENT: - oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg - extended_arg = 0 - i = i+2 - if op == EXTENDED_ARG: - extended_arg = oparg*65536L - if op in GLOBAL_OPS: - out_names.add(names[oparg]) - #print 'extracted', out_names, ' from ', names - return out_names + names = set(code.co_names) + if code.co_consts: # see if nested function have any global refs + for const in code.co_consts: + if type(const) is types.CodeType: + names |= CloudPickler.extract_code_globals(const) + return names def extract_func_data(self, func): """ @@ -354,10 +324,7 @@ def extract_func_data(self, func): # extract all global ref's func_global_refs = CloudPickler.extract_code_globals(code) - if code.co_consts: # see if nested function have any global refs - for const in code.co_consts: - if type(const) is types.CodeType and const.co_names: - func_global_refs = func_global_refs.union( CloudPickler.extract_code_globals(const)) + # process all variables referenced by global environment f_globals = {} for var in func_global_refs: @@ -396,6 +363,12 @@ def get_contents(cell): return (code, f_globals, defaults, closure, dct, base_globals) + def save_builtin_function(self, obj): + if obj.__module__ is "__builtin__": + return self.save_global(obj) + return self.save_function(obj) + dispatch[types.BuiltinFunctionType] = save_builtin_function + def save_global(self, obj, name=None, pack=struct.pack): write = self.write memo = self.memo @@ -435,7 +408,7 @@ def save_global(self, obj, name=None, pack=struct.pack): try: klass = getattr(themodule, name) except AttributeError, a: - #print themodule, name, obj, type(obj) + # print themodule, name, obj, type(obj) raise pickle.PicklingError("Can't pickle builtin %s" % obj) else: raise @@ -480,7 +453,6 @@ def save_global(self, obj, name=None, pack=struct.pack): write(pickle.GLOBAL + modname + '\n' + name + '\n') self.memoize(obj) dispatch[types.ClassType] = save_global - dispatch[types.BuiltinFunctionType] = save_global dispatch[types.TypeType] = save_global def save_instancemethod(self, obj): @@ -551,23 +523,39 @@ def save_property(self, obj): dispatch[property] = save_property def save_itemgetter(self, obj): - """itemgetter serializer (needed for namedtuple support) - a bit of a pain as we need to read ctypes internals""" - class ItemGetterType(ctypes.Structure): - _fields_ = PyObject_HEAD + [ - ('nitems', ctypes.c_size_t), - ('item', ctypes.py_object) - ] - - - obj = ctypes.cast(ctypes.c_void_p(id(obj)), ctypes.POINTER(ItemGetterType)).contents - return self.save_reduce(operator.itemgetter, - obj.item if obj.nitems > 1 else (obj.item,)) - - if PyObject_HEAD: + """itemgetter serializer (needed for namedtuple support)""" + class Dummy: + def __getitem__(self, item): + return item + items = obj(Dummy()) + if not isinstance(items, tuple): + items = (items, ) + return self.save_reduce(operator.itemgetter, items) + + if type(operator.itemgetter) is type: dispatch[operator.itemgetter] = save_itemgetter + def save_attrgetter(self, obj): + """attrgetter serializer""" + class Dummy(object): + def __init__(self, attrs, index=None): + self.attrs = attrs + self.index = index + def __getattribute__(self, item): + attrs = object.__getattribute__(self, "attrs") + index = object.__getattribute__(self, "index") + if index is None: + index = len(attrs) + attrs.append(item) + else: + attrs[index] = ".".join([attrs[index], item]) + return type(self)(attrs, index) + attrs = [] + obj(Dummy(attrs)) + return self.save_reduce(operator.attrgetter, tuple(attrs)) + if type(operator.attrgetter) is type: + dispatch[operator.attrgetter] = save_attrgetter def save_reduce(self, func, args, state=None, listitems=None, dictitems=None, obj=None): @@ -660,11 +648,11 @@ def save_file(self, obj): if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") - if obj.name == '': + if obj is sys.stdout: return self.save_reduce(getattr, (sys,'stdout'), obj=obj) - if obj.name == '': + if obj is sys.stderr: return self.save_reduce(getattr, (sys,'stderr'), obj=obj) - if obj.name == '': + if obj is sys.stdin: raise pickle.PicklingError("Cannot pickle standard input") if hasattr(obj, 'isatty') and obj.isatty(): raise pickle.PicklingError("Cannot pickle files that map to tty objects") @@ -873,8 +861,7 @@ def _genpartial(func, args, kwds): kwds = {} return partial(func, *args, **kwds) - -def _fill_function(func, globals, defaults, closure, dict): +def _fill_function(func, globals, defaults, dict): """ Fills in the rest of function data into the skeleton function object that were created via _make_skel_func(). """ @@ -882,49 +869,28 @@ def _fill_function(func, globals, defaults, closure, dict): func.func_defaults = defaults func.func_dict = dict - if len(closure) != len(func.func_closure): - raise pickle.UnpicklingError("closure lengths don't match up") - for i in range(len(closure)): - _change_cell_value(func.func_closure[i], closure[i]) - return func -def _make_skel_func(code, num_closures, base_globals = None): +def _make_cell(value): + return (lambda: value).func_closure[0] + +def _reconstruct_closure(values): + return tuple([_make_cell(v) for v in values]) + +def _make_skel_func(code, closures, base_globals = None): """ Creates a skeleton function object that contains just the provided code and the correct number of cells in func_closure. All other func attributes (e.g. func_globals) are empty. """ - #build closure (cells): - if not ctypes: - raise Exception('ctypes failed to import; cannot build function') - - cellnew = ctypes.pythonapi.PyCell_New - cellnew.restype = ctypes.py_object - cellnew.argtypes = (ctypes.py_object,) - dummy_closure = tuple(map(lambda i: cellnew(None), range(num_closures))) + closure = _reconstruct_closure(closures) if closures else None if base_globals is None: base_globals = {} base_globals['__builtins__'] = __builtins__ return types.FunctionType(code, base_globals, - None, None, dummy_closure) - -# this piece of opaque code is needed below to modify 'cell' contents -cell_changer_code = new.code( - 1, 1, 2, 0, - ''.join([ - chr(dis.opmap['LOAD_FAST']), '\x00\x00', - chr(dis.opmap['DUP_TOP']), - chr(dis.opmap['STORE_DEREF']), '\x00\x00', - chr(dis.opmap['RETURN_VALUE']) - ]), - (), (), ('newval',), '', 'cell_changer', 1, '', ('c',), () -) - -def _change_cell_value(cell, newval): - """ Changes the contents of 'cell' object to newval """ - return new.function(cell_changer_code, {}, None, (), (cell,))(newval) + None, None, closure) + """Constructors for 3rd party libraries Note: These can never be renamed due to client compatibility issues""" diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 22ab8d30c0ae3..15445abf67147 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -42,10 +42,6 @@ def worker(sock): """ Called by a worker process after the fork(). """ - # Redirect stdout to stderr - os.dup2(2, 1) - sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 - signal.signal(SIGHUP, SIG_DFL) signal.signal(SIGCHLD, SIG_DFL) signal.signal(SIGTERM, SIG_DFL) @@ -102,6 +98,7 @@ def manager(): listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() write_int(listen_port, sys.stdout) + sys.stdout.flush() def shutdown(code): signal.signal(SIGTERM, SIG_DFL) @@ -115,7 +112,6 @@ def handle_sigterm(*args): signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP # Initialization complete - sys.stdout.close() try: while True: try: diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 7b2710b913128..a5f9341e819a9 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -355,7 +355,8 @@ class PickleSerializer(FramedSerializer): def dumps(self, obj): return cPickle.dumps(obj, 2) - loads = cPickle.loads + def loads(self, obj): + return cPickle.loads(obj) class CloudPickleSerializer(PickleSerializer): @@ -374,8 +375,11 @@ class MarshalSerializer(FramedSerializer): This serializer is faster than PickleSerializer but supports fewer datatypes. """ - dumps = marshal.dumps - loads = marshal.loads + def dumps(self, obj): + return marshal.dumps(obj) + + def loads(self, obj): + return marshal.loads(obj) class AutoSerializer(FramedSerializer): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index bb84ebe72cb24..2e7c2750a8bb6 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,6 +31,7 @@ import time import zipfile import random +from platform import python_implementation if sys.version_info[:2] <= (2, 6): import unittest2 as unittest @@ -41,7 +42,8 @@ from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer +from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ + CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType @@ -168,15 +170,46 @@ def test_namedtuple(self): p2 = loads(dumps(p1, 2)) self.assertEquals(p1, p2) - -# Regression test for SPARK-3415 -class CloudPickleTest(unittest.TestCase): + def test_itemgetter(self): + from operator import itemgetter + ser = CloudPickleSerializer() + d = range(10) + getter = itemgetter(1) + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + getter = itemgetter(0, 3) + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + def test_attrgetter(self): + from operator import attrgetter + ser = CloudPickleSerializer() + + class C(object): + def __getattr__(self, item): + return item + d = C() + getter = attrgetter("a") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + getter = attrgetter("a", "b") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + d.e = C() + getter = attrgetter("e.a") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + getter = attrgetter("e.a", "e.b") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + # Regression test for SPARK-3415 def test_pickling_file_handles(self): - from pyspark.cloudpickle import dumps - from StringIO import StringIO - from pickle import load + ser = CloudPickleSerializer() out1 = sys.stderr - out2 = load(StringIO(dumps(out1))) + out2 = ser.loads(ser.dumps(out1)) self.assertEquals(out1, out2) @@ -861,8 +894,42 @@ def test_oldhadoop(self): conf=input_conf).collect()) self.assertEqual(old_dataset, dict_data) - @unittest.skipIf(sys.version_info[:2] <= (2, 6), "Skipped on 2.6 until SPARK-2951 is fixed") def test_newhadoop(self): + basepath = self.tempdir.name + data = [(1, ""), + (1, "a"), + (2, "bcdf")] + self.sc.parallelize(data).saveAsNewAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text") + result = sorted(self.sc.newAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text").collect()) + self.assertEqual(result, data) + + conf = { + "mapreduce.outputformat.class": + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.Text", + "mapred.output.dir": basepath + "/newdataset/" + } + self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf) + input_conf = {"mapred.input.dir": basepath + "/newdataset/"} + new_dataset = sorted(self.sc.newAPIHadoopRDD( + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + conf=input_conf).collect()) + self.assertEqual(new_dataset, data) + + @unittest.skipIf(sys.version_info[:2] <= (2, 6) or python_implementation() == "PyPy", + "Skipped on 2.6 and PyPy until SPARK-2951 is fixed") + def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays array_data = [(1, array('d')), diff --git a/python/run-tests b/python/run-tests index d98840de59d2c..a67e5a99fbdcc 100755 --- a/python/run-tests +++ b/python/run-tests @@ -85,6 +85,27 @@ run_test "pyspark/mllib/tests.py" run_test "pyspark/mllib/tree.py" run_test "pyspark/mllib/util.py" +# Try to test with PyPy +if [ $(which pypy) ]; then + export PYSPARK_PYTHON="pypy" + echo "Testing with PyPy version:" + $PYSPARK_PYTHON --version + + run_test "pyspark/rdd.py" + run_test "pyspark/context.py" + run_test "pyspark/conf.py" + run_test "pyspark/sql.py" + # These tests are included in the module-level docs, and so must + # be handled on a higher level rather than within the python file. + export PYSPARK_DOC_TEST=1 + run_test "pyspark/broadcast.py" + run_test "pyspark/accumulators.py" + run_test "pyspark/serializers.py" + unset PYSPARK_DOC_TEST + run_test "pyspark/shuffle.py" + run_test "pyspark/tests.py" +fi + if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green echo "Tests passed." From 885d1621bc06bc1f009c9707c3452eac26baf828 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 12 Sep 2014 19:05:39 -0700 Subject: [PATCH 55/76] [SPARK-3500] [SQL] use JavaSchemaRDD as SchemaRDD._jschema_rdd Currently, SchemaRDD._jschema_rdd is SchemaRDD, the Scala API (coalesce(), repartition()) can not been called in Python easily, there is no way to specify the implicit parameter `ord`. The _jrdd is an JavaRDD, so _jschema_rdd should also be JavaSchemaRDD. In this patch, change _schema_rdd to JavaSchemaRDD, also added an assert for it. If some methods are missing from JavaSchemaRDD, then it's called by _schema_rdd.baseSchemaRDD().xxx(). BTW, Do we need JavaSQLContext? Author: Davies Liu Closes #2369 from davies/fix_schemardd and squashes the following commits: abee159 [Davies Liu] use JavaSchemaRDD as SchemaRDD._jschema_rdd --- python/pyspark/sql.py | 38 ++++++++++++++++++-------------------- python/pyspark/tests.py | 28 ++++++++++++++++++++++++++++ 2 files changed, 46 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 53eea6d6cf3ba..fc9310fef318c 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1122,7 +1122,7 @@ def applySchema(self, rdd, schema): batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) jrdd = self._pythonToJava(rdd._jrdd, batched) srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), str(schema)) - return SchemaRDD(srdd, self) + return SchemaRDD(srdd.toJavaSchemaRDD(), self) def registerRDDAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. @@ -1134,8 +1134,8 @@ def registerRDDAsTable(self, rdd, tableName): >>> sqlCtx.registerRDDAsTable(srdd, "table1") """ if (rdd.__class__ is SchemaRDD): - jschema_rdd = rdd._jschema_rdd - self._ssql_ctx.registerRDDAsTable(jschema_rdd, tableName) + srdd = rdd._jschema_rdd.baseSchemaRDD() + self._ssql_ctx.registerRDDAsTable(srdd, tableName) else: raise ValueError("Can only register SchemaRDD as table") @@ -1151,7 +1151,7 @@ def parquetFile(self, path): >>> sorted(srdd.collect()) == sorted(srdd2.collect()) True """ - jschema_rdd = self._ssql_ctx.parquetFile(path) + jschema_rdd = self._ssql_ctx.parquetFile(path).toJavaSchemaRDD() return SchemaRDD(jschema_rdd, self) def jsonFile(self, path, schema=None): @@ -1207,11 +1207,11 @@ def jsonFile(self, path, schema=None): [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] """ if schema is None: - jschema_rdd = self._ssql_ctx.jsonFile(path) + srdd = self._ssql_ctx.jsonFile(path) else: scala_datatype = self._ssql_ctx.parseDataType(str(schema)) - jschema_rdd = self._ssql_ctx.jsonFile(path, scala_datatype) - return SchemaRDD(jschema_rdd, self) + srdd = self._ssql_ctx.jsonFile(path, scala_datatype) + return SchemaRDD(srdd.toJavaSchemaRDD(), self) def jsonRDD(self, rdd, schema=None): """Loads an RDD storing one JSON object per string as a L{SchemaRDD}. @@ -1275,11 +1275,11 @@ def func(iterator): keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) if schema is None: - jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + srdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) else: scala_datatype = self._ssql_ctx.parseDataType(str(schema)) - jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) - return SchemaRDD(jschema_rdd, self) + srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) + return SchemaRDD(srdd.toJavaSchemaRDD(), self) def sql(self, sqlQuery): """Return a L{SchemaRDD} representing the result of the given query. @@ -1290,7 +1290,7 @@ def sql(self, sqlQuery): >>> srdd2.collect() [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ - return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self) + return SchemaRDD(self._ssql_ctx.sql(sqlQuery).toJavaSchemaRDD(), self) def table(self, tableName): """Returns the specified table as a L{SchemaRDD}. @@ -1301,7 +1301,7 @@ def table(self, tableName): >>> sorted(srdd.collect()) == sorted(srdd2.collect()) True """ - return SchemaRDD(self._ssql_ctx.table(tableName), self) + return SchemaRDD(self._ssql_ctx.table(tableName).toJavaSchemaRDD(), self) def cacheTable(self, tableName): """Caches the specified table in-memory.""" @@ -1353,7 +1353,7 @@ def hiveql(self, hqlQuery): warnings.warn("hiveql() is deprecated as the sql function now parses using HiveQL by" + "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'", DeprecationWarning) - return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery), self) + return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery).toJavaSchemaRDD(), self) def hql(self, hqlQuery): """ @@ -1524,6 +1524,8 @@ class SchemaRDD(RDD): def __init__(self, jschema_rdd, sql_ctx): self.sql_ctx = sql_ctx self._sc = sql_ctx._sc + clsName = jschema_rdd.getClass().getName() + assert clsName.endswith("JavaSchemaRDD"), "jschema_rdd must be JavaSchemaRDD" self._jschema_rdd = jschema_rdd self._id = None self.is_cached = False @@ -1540,7 +1542,7 @@ def _jrdd(self): L{pyspark.rdd.RDD} super class (map, filter, etc.). """ if not hasattr(self, '_lazy_jrdd'): - self._lazy_jrdd = self._jschema_rdd.javaToPython() + self._lazy_jrdd = self._jschema_rdd.baseSchemaRDD().javaToPython() return self._lazy_jrdd def id(self): @@ -1598,7 +1600,7 @@ def saveAsTable(self, tableName): def schema(self): """Returns the schema of this SchemaRDD (represented by a L{StructType}).""" - return _parse_datatype_string(self._jschema_rdd.schema().toString()) + return _parse_datatype_string(self._jschema_rdd.baseSchemaRDD().schema().toString()) def schemaString(self): """Returns the output schema in the tree format.""" @@ -1649,8 +1651,6 @@ def mapPartitionsWithIndex(self, f, preservesPartitioning=False): rdd = RDD(self._jrdd, self._sc, self._jrdd_deserializer) schema = self.schema() - import pickle - pickle.loads(pickle.dumps(schema)) def applySchema(_, it): cls = _create_cls(schema) @@ -1687,10 +1687,8 @@ def isCheckpointed(self): def getCheckpointFile(self): checkpointFile = self._jschema_rdd.getCheckpointFile() - if checkpointFile.isDefined(): + if checkpointFile.isPresent(): return checkpointFile.get() - else: - return None def coalesce(self, numPartitions, shuffle=False): rdd = self._jschema_rdd.coalesce(numPartitions, shuffle) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 2e7c2750a8bb6..b687d695b01c4 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -607,6 +607,34 @@ def test_broadcast_in_udf(self): [res] = self.sqlCtx.sql("SELECT MYUDF('')").collect() self.assertEqual("", res[0]) + def test_basic_functions(self): + rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) + srdd = self.sqlCtx.jsonRDD(rdd) + srdd.count() + srdd.collect() + srdd.schemaString() + srdd.schema() + + # cache and checkpoint + self.assertFalse(srdd.is_cached) + srdd.persist() + srdd.unpersist() + srdd.cache() + self.assertTrue(srdd.is_cached) + self.assertFalse(srdd.isCheckpointed()) + self.assertEqual(None, srdd.getCheckpointFile()) + + srdd = srdd.coalesce(2, True) + srdd = srdd.repartition(3) + srdd = srdd.distinct() + srdd.intersection(srdd) + self.assertEqual(2, srdd.count()) + + srdd.registerTempTable("temp") + srdd = self.sqlCtx.sql("select foo from temp") + srdd.count() + srdd.collect() + class TestIO(PySparkTestCase): From 6d887db7891be643f0131b136e82191b5f6eb407 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 12 Sep 2014 20:14:09 -0700 Subject: [PATCH 56/76] [SPARK-3515][SQL] Moves test suite setup code to beforeAll rather than in constructor Please refer to the JIRA ticket for details. **NOTE** We should check all test suites that do similar initialization-like side effects in their constructors. This PR only fixes `ParquetMetastoreSuite` because it breaks our Jenkins Maven build. Author: Cheng Lian Closes #2375 from liancheng/say-no-to-constructor and squashes the following commits: 0ceb75b [Cheng Lian] Moves test suite setup code to beforeAll rather than in constructor --- .../sql/parquet/ParquetMetastoreSuite.scala | 53 +++++++++---------- 1 file changed, 24 insertions(+), 29 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala index 0723be7298e15..e380280f301c1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -20,14 +20,10 @@ package org.apache.spark.sql.parquet import java.io.File -import org.apache.spark.sql.hive.execution.HiveTableScan import org.scalatest.BeforeAndAfterAll -import scala.reflect.ClassTag - -import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive._ case class ParquetData(intField: Int, stringField: String) @@ -36,27 +32,19 @@ case class ParquetData(intField: Int, stringField: String) * Tests for our SerDe -> Native parquet scan conversion. */ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { - override def beforeAll(): Unit = { - setConf("spark.sql.hive.convertMetastoreParquet", "true") - } - - override def afterAll(): Unit = { - setConf("spark.sql.hive.convertMetastoreParquet", "false") - } - - val partitionedTableDir = File.createTempFile("parquettests", "sparksql") - partitionedTableDir.delete() - partitionedTableDir.mkdir() - - (1 to 10).foreach { p => - val partDir = new File(partitionedTableDir, s"p=$p") - sparkContext.makeRDD(1 to 10) - .map(i => ParquetData(i, s"part-$p")) - .saveAsParquetFile(partDir.getCanonicalPath) - } - - sql(s""" + val partitionedTableDir = File.createTempFile("parquettests", "sparksql") + partitionedTableDir.delete() + partitionedTableDir.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDir, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetData(i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + + sql(s""" create external table partitioned_parquet ( intField INT, @@ -70,7 +58,7 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { location '${partitionedTableDir.getCanonicalPath}' """) - sql(s""" + sql(s""" create external table normal_parquet ( intField INT, @@ -83,8 +71,15 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' """) - (1 to 10).foreach { p => - sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") + } + + setConf("spark.sql.hive.convertMetastoreParquet", "true") + } + + override def afterAll(): Unit = { + setConf("spark.sql.hive.convertMetastoreParquet", "false") } test("project the partitioning column") { From 2584ea5b23b1c5a4df9549b94bfc9b8e0900532e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 12 Sep 2014 21:55:39 -0700 Subject: [PATCH 57/76] [SPARK-3469] Make sure all TaskCompletionListener are called even with failures This is necessary because we rely on this callback interface to clean resources up. The old behavior would lead to resource leaks. Note that this also changes the fault semantics of TaskCompletionListener. Previously failures in TaskCompletionListeners would result in the task being reported immediately. With this change, we report the exception at the end, and the reported exception is a TaskCompletionListenerException that contains all the exception messages. Author: Reynold Xin Closes #2343 from rxin/taskcontext-callback and squashes the following commits: a3845b2 [Reynold Xin] Mark TaskCompletionListenerException as private[spark]. ac5baea [Reynold Xin] Removed obsolete comment. aa68ea4 [Reynold Xin] Throw an exception if task completion callback fails. 29b6162 [Reynold Xin] oops compilation failed. 1cb444d [Reynold Xin] [SPARK-3469] Call all TaskCompletionListeners even if some fail. --- .../scala/org/apache/spark/TaskContext.scala | 18 ++++++++-- .../TaskCompletionListenerException.scala | 34 +++++++++++++++++++ .../spark/scheduler/TaskContextSuite.scala | 22 ++++++++++-- 3 files changed, 69 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 2b99b8a5af250..51b3e4d5e0936 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.TaskCompletionListener +import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener} /** @@ -41,7 +41,7 @@ class TaskContext( val attemptId: Long, val runningLocally: Boolean = false, private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty) - extends Serializable { + extends Serializable with Logging { @deprecated("use partitionId", "0.8.1") def splitId = partitionId @@ -103,8 +103,20 @@ class TaskContext( /** Marks the task as completed and triggers the listeners. */ private[spark] def markTaskCompleted(): Unit = { completed = true + val errorMsgs = new ArrayBuffer[String](2) // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach { _.onTaskCompletion(this) } + onCompleteCallbacks.reverse.foreach { listener => + try { + listener.onTaskCompletion(this) + } catch { + case e: Throwable => + errorMsgs += e.getMessage + logError("Error in TaskCompletionListener", e) + } + } + if (errorMsgs.nonEmpty) { + throw new TaskCompletionListenerException(errorMsgs) + } } /** Marks the task for interruption, i.e. cancellation. */ diff --git a/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala b/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala new file mode 100644 index 0000000000000..f64e069cd1724 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * Exception thrown when there is an exception in + * executing the callback in TaskCompletionListener. + */ +private[spark] +class TaskCompletionListenerException(errorMessages: Seq[String]) extends Exception { + + override def getMessage: String = { + if (errorMessages.size == 1) { + errorMessages.head + } else { + errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n") + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index db2ad829a48f9..faba5508c906c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.scheduler +import org.mockito.Mockito._ +import org.mockito.Matchers.any + import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.Utils +import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener} + class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - test("Calls executeOnCompleteCallbacks after failure") { + test("calls TaskCompletionListener after failure") { TaskContextSuite.completed = false sc = new SparkContext("local", "test") val rdd = new RDD[String](sc, List()) { @@ -45,6 +49,20 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte } assert(TaskContextSuite.completed === true) } + + test("all TaskCompletionListeners should be called even if some fail") { + val context = new TaskContext(0, 0, 0) + val listener = mock(classOf[TaskCompletionListener]) + context.addTaskCompletionListener(_ => throw new Exception("blah")) + context.addTaskCompletionListener(listener) + context.addTaskCompletionListener(_ => throw new Exception("blah")) + + intercept[TaskCompletionListenerException] { + context.markTaskCompleted() + } + + verify(listener, times(1)).onTaskCompletion(any()) + } } private object TaskContextSuite { From e11eeb71fa3a5fe7ddacb94d5b93b173d4d901a8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 12 Sep 2014 21:58:02 -0700 Subject: [PATCH 58/76] [SQL][Docs] Update SQL programming guide to show the correct default value of containsNull in an ArrayType After #1889, the default value of `containsNull` in an `ArrayType` is `true`. Author: Yin Huai Closes #2374 from yhuai/containsNull and squashes the following commits: dc609a3 [Yin Huai] Update the SQL programming guide to show the correct default value of containsNull in an ArrayType (the default value is true instead of false). --- docs/sql-programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d83efa4bab324..3159d52787d5a 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1232,7 +1232,7 @@ import org.apache.spark.sql._ scala.collection.Seq ArrayType(elementType, [containsNull])
- Note: The default value of containsNull is false. + Note: The default value of containsNull is true. @@ -1358,7 +1358,7 @@ please use factory methods provided in java.util.List DataType.createArrayType(elementType)
- Note: The value of containsNull will be false
+ Note: The value of containsNull will be true
DataType.createArrayType(elementType, containsNull). @@ -1505,7 +1505,7 @@ from pyspark.sql import * list, tuple, or array ArrayType(elementType, [containsNull])
- Note: The default value of containsNull is False. + Note: The default value of containsNull is True. From feaa3706f17e44efcdac9f0a543a5b91232771ce Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 12 Sep 2014 22:50:37 -0700 Subject: [PATCH 59/76] SPARK-3470 [CORE] [STREAMING] Add Closeable / close() to Java context objects ... that expose a stop() lifecycle method. This doesn't add `AutoCloseable`, which is Java 7+ only. But it should be possible to use try-with-resources on a `Closeable` in Java 7, as long as the `close()` does not throw a checked exception, and these don't. Q.E.D. Author: Sean Owen Closes #2346 from srowen/SPARK-3470 and squashes the following commits: 612c21d [Sean Owen] Add Closeable / close() to Java context objects that expose a stop() lifecycle method --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 7 ++++++- .../spark/streaming/api/java/JavaStreamingContext.scala | 7 +++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8e178bc8480f7..23f7e6be81a90 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -17,6 +17,7 @@ package org.apache.spark.api.java +import java.io.Closeable import java.util import java.util.{Map => JMap} @@ -40,7 +41,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ -class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { +class JavaSparkContext(val sc: SparkContext) + extends JavaSparkContextVarargsWorkaround with Closeable { + /** * Create a JavaSparkContext that loads settings from system properties (for instance, when * launching with ./bin/spark-submit). @@ -534,6 +537,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.stop() } + override def close(): Unit = stop() + /** * Get Spark's home location from either a value set through the constructor, * or the spark.home Java property, or the SPARK_HOME environment variable diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 18605cac7006c..9dc26dc6b32a1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -21,7 +21,7 @@ package org.apache.spark.streaming.api.java import scala.collection.JavaConversions._ import scala.reflect.ClassTag -import java.io.InputStream +import java.io.{Closeable, InputStream} import java.util.{List => JList, Map => JMap} import akka.actor.{Props, SupervisorStrategy} @@ -49,7 +49,7 @@ import org.apache.spark.streaming.receiver.Receiver * respectively. `context.awaitTransformation()` allows the current thread to wait for the * termination of a context by `stop()` or by an exception. */ -class JavaStreamingContext(val ssc: StreamingContext) { +class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Create a StreamingContext. @@ -540,6 +540,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { def stop(stopSparkContext: Boolean, stopGracefully: Boolean) = { ssc.stop(stopSparkContext, stopGracefully) } + + override def close(): Unit = stop() + } /** From b4dded40fbecb485f1ddfd8316b44d42a1554d64 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 12 Sep 2014 22:51:25 -0700 Subject: [PATCH 60/76] Proper indent for the previous commit. --- .../main/scala/org/apache/spark/api/java/JavaSparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 23f7e6be81a90..791d853a015a1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -42,7 +42,7 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) - extends JavaSparkContextVarargsWorkaround with Closeable { + extends JavaSparkContextVarargsWorkaround with Closeable { /** * Create a JavaSparkContext that loads settings from system properties (for instance, when From a523ceaf159733dabcef84c7adc1463546679f65 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 13 Sep 2014 12:34:20 -0700 Subject: [PATCH 61/76] [SQL] [Docs] typo fixes * Fixed random typo * Added in missing description for DecimalType Author: Nicholas Chammas Closes #2367 from nchammas/patch-1 and squashes the following commits: aa528be [Nicholas Chammas] doc fix for SQL DecimalType 3247ac1 [Nicholas Chammas] [SQL] [Docs] typo fixes --- docs/sql-programming-guide.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3159d52787d5a..8d41fdec699e9 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -918,7 +918,6 @@ options. ## Migration Guide for Shark User ### Scheduling -s To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: @@ -1110,7 +1109,7 @@ evaluated by the SQL execution engine. A full list of the functions supported c The range of numbers is from `-9223372036854775808` to `9223372036854775807`. - `FloatType`: Represents 4-byte single-precision floating point numbers. - `DoubleType`: Represents 8-byte double-precision floating point numbers. - - `DecimalType`: + - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. * String type - `StringType`: Represents character string values. * Binary type From 184cd51c4207c23726da97f907f2d912a5a44845 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 13 Sep 2014 12:35:40 -0700 Subject: [PATCH 62/76] [SPARK-3481][SQL] Removes the evil MINOR HACK This is a follow up of #2352. Now we can finally remove the evil "MINOR HACK", which covered up the eldest bug in the history of Spark SQL (see details [here](https://github.com/apache/spark/pull/2352#issuecomment-55440621)). Author: Cheng Lian Closes #2377 from liancheng/remove-evil-minor-hack and squashes the following commits: 0869c78 [Cheng Lian] Removes the evil MINOR HACK --- .../spark/sql/hive/execution/HiveComparisonTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 671c3b162f875..79cc7a3fcc7d6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -250,9 +250,9 @@ abstract class HiveComparisonTest } try { - // MINOR HACK: You must run a query before calling reset the first time. - TestHive.sql("SHOW TABLES") - if (reset) { TestHive.reset() } + if (reset) { + TestHive.reset() + } val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => From 74049249abb952ad061c0e221c22ff894a9e9c8d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 13 Sep 2014 15:08:30 -0700 Subject: [PATCH 63/76] [SPARK-3294][SQL] Eliminates boxing costs from in-memory columnar storage This is a major refactoring of the in-memory columnar storage implementation, aims to eliminate boxing costs from critical paths (building/accessing column buffers) as much as possible. The basic idea is to refactor all major interfaces into a row-based form and use them together with `SpecificMutableRow`. The difficult part is how to adapt all compression schemes, esp. `RunLengthEncoding` and `DictionaryEncoding`, to this design. Since in-memory compression is disabled by default for now, and this PR should be strictly better than before no matter in-memory compression is enabled or not, maybe I'll finish that part in another PR. **UPDATE** This PR also took the chance to optimize `HiveTableScan` by 1. leveraging `SpecificMutableRow` to avoid boxing cost, and 1. building specific `Writable` unwrapper functions a head of time to avoid per row pattern matching and branching costs. TODO - [x] Benchmark - [ ] ~~Eliminate boxing costs in `RunLengthEncoding`~~ (left to future PRs) - [ ] ~~Eliminate boxing costs in `DictionaryEncoding` (seems not easy to do without specializing `DictionaryEncoding` for every supported column type)~~ (left to future PRs) ## Micro benchmark The benchmark uses a 10 million line CSV table consists of bytes, shorts, integers, longs, floats and doubles, measures the time to build the in-memory version of this table, and the time to scan the whole in-memory table. Benchmark code can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-hivetablescanbenchmark-scala). Script used to generate the input table can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-tablegen-scala). Speedup: - Hive table scanning + column buffer building: **18.74%** The original benchmark uses 1K as in-memory batch size, when increased to 10K, it can be 28.32% faster. - In-memory table scanning: **7.95%** Before: | Building | Scanning ------- | -------- | -------- 1 | 16472 | 525 2 | 16168 | 530 3 | 16386 | 529 4 | 16184 | 538 5 | 16209 | 521 Average | 16283.8 | 528.6 After: | Building | Scanning ------- | -------- | -------- 1 | 13124 | 458 2 | 13260 | 529 3 | 12981 | 463 4 | 13214 | 483 5 | 13583 | 500 Average | 13232.4 | 486.6 Author: Cheng Lian Closes #2327 from liancheng/prevent-boxing/unboxing and squashes the following commits: 4419fe4 [Cheng Lian] Addressing comments e5d2cf2 [Cheng Lian] Bug fix: should call setNullAt when field value is null to avoid NPE 8b8552b [Cheng Lian] Only checks for partition batch pruning flag once 489f97b [Cheng Lian] Bug fix: TableReader.fillObject uses wrong ordinals 97bbc4e [Cheng Lian] Optimizes hive.TableReader by by providing specific Writable unwrappers a head of time 3dc1f94 [Cheng Lian] Minor changes to eliminate row object creation 5b39cb9 [Cheng Lian] Lowers log level of compression scheme details f2a7890 [Cheng Lian] Use SpecificMutableRow in InMemoryColumnarTableScan to avoid boxing 9cf30b0 [Cheng Lian] Added row based ColumnType.append/extract 456c366 [Cheng Lian] Made compression decoder row based edac3cd [Cheng Lian] Makes ColumnAccessor.extractSingle row based 8216936 [Cheng Lian] Removes boxing cost in IntDelta and LongDelta by providing specialized implementations b70d519 [Cheng Lian] Made some in-memory columnar storage interfaces row-based --- .../catalyst/expressions/SpecificRow.scala | 2 +- .../spark/sql/columnar/ColumnAccessor.scala | 8 +- .../spark/sql/columnar/ColumnBuilder.scala | 27 +- .../spark/sql/columnar/ColumnStats.scala | 16 +- .../spark/sql/columnar/ColumnType.scala | 178 ++++++++++-- .../columnar/InMemoryColumnarTableScan.scala | 92 +++--- .../sql/columnar/NullableColumnAccessor.scala | 4 +- .../sql/columnar/NullableColumnBuilder.scala | 8 +- .../CompressibleColumnAccessor.scala | 7 +- .../CompressibleColumnBuilder.scala | 24 +- .../compression/CompressionScheme.scala | 16 +- .../compression/compressionSchemes.scala | 264 +++++++++++------- .../spark/sql/columnar/ColumnStatsSuite.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 11 +- .../columnar/InMemoryColumnarQuerySuite.scala | 2 +- .../NullableColumnAccessorSuite.scala | 4 +- .../columnar/NullableColumnBuilderSuite.scala | 4 +- .../columnar/PartitionBatchPruningSuite.scala | 6 +- .../compression/BooleanBitSetSuite.scala | 7 +- .../compression/DictionaryEncodingSuite.scala | 9 +- .../compression/IntegralDeltaSuite.scala | 9 +- .../compression/RunLengthEncodingSuite.scala | 9 +- .../apache/spark/sql/hive/TableReader.scala | 119 ++++---- .../sql/hive/execution/HiveQuerySuite.scala | 18 +- 24 files changed, 554 insertions(+), 292 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala index 088f11ee4aa53..9cbab3d5d0d0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -171,7 +171,7 @@ final class MutableByte extends MutableValue { } final class MutableAny extends MutableValue { - var value: Any = 0 + var value: Any = _ def boxed = if (isNull) null else value def update(v: Any) = value = { isNull = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 42a5a9a84f362..c9faf0852142a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -50,11 +50,13 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( def hasNext = buffer.hasRemaining - def extractTo(row: MutableRow, ordinal: Int) { - columnType.setField(row, ordinal, extractSingle(buffer)) + def extractTo(row: MutableRow, ordinal: Int): Unit = { + extractSingle(row, ordinal) } - def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer) + def extractSingle(row: MutableRow, ordinal: Int): Unit = { + columnType.extract(buffer, row, ordinal) + } protected def underlyingBuffer = buffer } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index b3ec5ded22422..2e61a981375aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -68,10 +68,9 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId) } - override def appendFrom(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - buffer = ensureFreeSpace(buffer, columnType.actualSize(field)) - columnType.append(field, buffer) + override def appendFrom(row: Row, ordinal: Int): Unit = { + buffer = ensureFreeSpace(buffer, columnType.actualSize(row, ordinal)) + columnType.append(row, ordinal, buffer) } override def build() = { @@ -142,16 +141,16 @@ private[sql] object ColumnBuilder { useCompression: Boolean = false): ColumnBuilder = { val builder = (typeId match { - case INT.typeId => new IntColumnBuilder - case LONG.typeId => new LongColumnBuilder - case FLOAT.typeId => new FloatColumnBuilder - case DOUBLE.typeId => new DoubleColumnBuilder - case BOOLEAN.typeId => new BooleanColumnBuilder - case BYTE.typeId => new ByteColumnBuilder - case SHORT.typeId => new ShortColumnBuilder - case STRING.typeId => new StringColumnBuilder - case BINARY.typeId => new BinaryColumnBuilder - case GENERIC.typeId => new GenericColumnBuilder + case INT.typeId => new IntColumnBuilder + case LONG.typeId => new LongColumnBuilder + case FLOAT.typeId => new FloatColumnBuilder + case DOUBLE.typeId => new DoubleColumnBuilder + case BOOLEAN.typeId => new BooleanColumnBuilder + case BYTE.typeId => new ByteColumnBuilder + case SHORT.typeId => new ShortColumnBuilder + case STRING.typeId => new StringColumnBuilder + case BINARY.typeId => new BinaryColumnBuilder + case GENERIC.typeId => new GenericColumnBuilder case TIMESTAMP.typeId => new TimestampColumnBuilder }).asInstanceOf[ColumnBuilder] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index fc343ccb995c2..203a714e03c97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -69,7 +69,7 @@ private[sql] class ByteColumnStats extends ColumnStats { var lower = Byte.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getByte(ordinal) if (value > upper) upper = value @@ -87,7 +87,7 @@ private[sql] class ShortColumnStats extends ColumnStats { var lower = Short.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getShort(ordinal) if (value > upper) upper = value @@ -105,7 +105,7 @@ private[sql] class LongColumnStats extends ColumnStats { var lower = Long.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getLong(ordinal) if (value > upper) upper = value @@ -123,7 +123,7 @@ private[sql] class DoubleColumnStats extends ColumnStats { var lower = Double.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getDouble(ordinal) if (value > upper) upper = value @@ -141,7 +141,7 @@ private[sql] class FloatColumnStats extends ColumnStats { var lower = Float.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getFloat(ordinal) if (value > upper) upper = value @@ -159,7 +159,7 @@ private[sql] class IntColumnStats extends ColumnStats { var lower = Int.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getInt(ordinal) if (value > upper) upper = value @@ -177,7 +177,7 @@ private[sql] class StringColumnStats extends ColumnStats { var lower: String = null var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getString(ordinal) if (upper == null || value.compareTo(upper) > 0) upper = value @@ -195,7 +195,7 @@ private[sql] class TimestampColumnStats extends ColumnStats { var lower: Timestamp = null var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row(ordinal).asInstanceOf[Timestamp] if (upper == null || value.compareTo(upper) > 0) upper = value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 9a61600115872..198b5756676aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import java.sql.Timestamp import scala.reflect.runtime.universe.TypeTag -import java.sql.Timestamp - import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ @@ -46,16 +45,33 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( */ def extract(buffer: ByteBuffer): JvmType + /** + * Extracts a value out of the buffer at the buffer's current position and stores in + * `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs whenever + * possible. + */ + def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + setField(row, ordinal, extract(buffer)) + } + /** * Appends the given value v of type T into the given ByteBuffer. */ - def append(v: JvmType, buffer: ByteBuffer) + def append(v: JvmType, buffer: ByteBuffer): Unit + + /** + * Appends `row(ordinal)` of type T into the given ByteBuffer. Subclasses should override this + * method to avoid boxing/unboxing costs whenever possible. + */ + def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + append(getField(row, ordinal), buffer) + } /** - * Returns the size of the value. This is used to calculate the size of variable length types - * such as byte arrays and strings. + * Returns the size of the value `row(ordinal)`. This is used to calculate the size of variable + * length types such as byte arrays and strings. */ - def actualSize(v: JvmType): Int = defaultSize + def actualSize(row: Row, ordinal: Int): Int = defaultSize /** * Returns `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs @@ -67,7 +83,15 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( * Sets `row(ordinal)` to `field`. Subclasses should override this method to avoid boxing/unboxing * costs whenever possible. */ - def setField(row: MutableRow, ordinal: Int, value: JvmType) + def setField(row: MutableRow, ordinal: Int, value: JvmType): Unit + + /** + * Copies `from(fromOrdinal)` to `to(toOrdinal)`. Subclasses should override this method to avoid + * boxing/unboxing costs whenever possible. + */ + def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to(toOrdinal) = from(fromOrdinal) + } /** * Creates a duplicated copy of the value. @@ -90,119 +114,205 @@ private[sql] abstract class NativeColumnType[T <: NativeType]( } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { - def append(v: Int, buffer: ByteBuffer) { + def append(v: Int, buffer: ByteBuffer): Unit = { buffer.putInt(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putInt(row.getInt(ordinal)) + } + def extract(buffer: ByteBuffer) = { buffer.getInt() } - override def setField(row: MutableRow, ordinal: Int, value: Int) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setInt(ordinal, buffer.getInt()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Int): Unit = { row.setInt(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getInt(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setInt(toOrdinal, from.getInt(fromOrdinal)) + } } private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { - override def append(v: Long, buffer: ByteBuffer) { + override def append(v: Long, buffer: ByteBuffer): Unit = { buffer.putLong(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putLong(row.getLong(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getLong() } - override def setField(row: MutableRow, ordinal: Int, value: Long) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setLong(ordinal, buffer.getLong()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Long): Unit = { row.setLong(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getLong(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setLong(toOrdinal, from.getLong(fromOrdinal)) + } } private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { - override def append(v: Float, buffer: ByteBuffer) { + override def append(v: Float, buffer: ByteBuffer): Unit = { buffer.putFloat(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putFloat(row.getFloat(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getFloat() } - override def setField(row: MutableRow, ordinal: Int, value: Float) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setFloat(ordinal, buffer.getFloat()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Float): Unit = { row.setFloat(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getFloat(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setFloat(toOrdinal, from.getFloat(fromOrdinal)) + } } private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { - override def append(v: Double, buffer: ByteBuffer) { + override def append(v: Double, buffer: ByteBuffer): Unit = { buffer.putDouble(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putDouble(row.getDouble(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getDouble() } - override def setField(row: MutableRow, ordinal: Int, value: Double) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setDouble(ordinal, buffer.getDouble()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Double): Unit = { row.setDouble(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getDouble(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setDouble(toOrdinal, from.getDouble(fromOrdinal)) + } } private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { - override def append(v: Boolean, buffer: ByteBuffer) { - buffer.put(if (v) 1.toByte else 0.toByte) + override def append(v: Boolean, buffer: ByteBuffer): Unit = { + buffer.put(if (v) 1: Byte else 0: Byte) + } + + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.put(if (row.getBoolean(ordinal)) 1: Byte else 0: Byte) } override def extract(buffer: ByteBuffer) = buffer.get() == 1 - override def setField(row: MutableRow, ordinal: Int, value: Boolean) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setBoolean(ordinal, buffer.get() == 1) + } + + override def setField(row: MutableRow, ordinal: Int, value: Boolean): Unit = { row.setBoolean(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getBoolean(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setBoolean(toOrdinal, from.getBoolean(fromOrdinal)) + } } private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { - override def append(v: Byte, buffer: ByteBuffer) { + override def append(v: Byte, buffer: ByteBuffer): Unit = { buffer.put(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.put(row.getByte(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.get() } - override def setField(row: MutableRow, ordinal: Int, value: Byte) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setByte(ordinal, buffer.get()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Byte): Unit = { row.setByte(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getByte(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setByte(toOrdinal, from.getByte(fromOrdinal)) + } } private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { - override def append(v: Short, buffer: ByteBuffer) { + override def append(v: Short, buffer: ByteBuffer): Unit = { buffer.putShort(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putShort(row.getShort(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getShort() } - override def setField(row: MutableRow, ordinal: Int, value: Short) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setShort(ordinal, buffer.getShort()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Short): Unit = { row.setShort(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getShort(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setShort(toOrdinal, from.getShort(fromOrdinal)) + } } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { - override def actualSize(v: String): Int = v.getBytes("utf-8").length + 4 + override def actualSize(row: Row, ordinal: Int): Int = { + row.getString(ordinal).getBytes("utf-8").length + 4 + } - override def append(v: String, buffer: ByteBuffer) { + override def append(v: String, buffer: ByteBuffer): Unit = { val stringBytes = v.getBytes("utf-8") buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } @@ -214,11 +324,15 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { new String(stringBytes, "utf-8") } - override def setField(row: MutableRow, ordinal: Int, value: String) { + override def setField(row: MutableRow, ordinal: Int, value: String): Unit = { row.setString(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getString(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setString(toOrdinal, from.getString(fromOrdinal)) + } } private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { @@ -228,7 +342,7 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { timestamp } - override def append(v: Timestamp, buffer: ByteBuffer) { + override def append(v: Timestamp, buffer: ByteBuffer): Unit = { buffer.putLong(v.getTime).putInt(v.getNanos) } @@ -236,7 +350,7 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { row(ordinal).asInstanceOf[Timestamp] } - override def setField(row: MutableRow, ordinal: Int, value: Timestamp) { + override def setField(row: MutableRow, ordinal: Int, value: Timestamp): Unit = { row(ordinal) = value } } @@ -246,9 +360,11 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( defaultSize: Int) extends ColumnType[T, Array[Byte]](typeId, defaultSize) { - override def actualSize(v: Array[Byte]) = v.length + 4 + override def actualSize(row: Row, ordinal: Int) = { + getField(row, ordinal).length + 4 + } - override def append(v: Array[Byte], buffer: ByteBuffer) { + override def append(v: Array[Byte], buffer: ByteBuffer): Unit = { buffer.putInt(v.length).put(v, 0, v.length) } @@ -261,7 +377,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { - override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = value } @@ -272,7 +388,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. private[sql] object GENERIC extends ByteArrayColumnType[DataType](10, 16) { - override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 6eab2f23c18e1..8a3612cdf19be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -52,7 +52,7 @@ private[sql] case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { val output = child.output - val cached = child.execute().mapPartitions { baseIterator => + val cached = child.execute().mapPartitions { rowIterator => new Iterator[CachedBatch] { def next() = { val columnBuilders = output.map { attribute => @@ -61,11 +61,9 @@ private[sql] case class InMemoryRelation( ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) }.toArray - var row: Row = null var rowCount = 0 - - while (baseIterator.hasNext && rowCount < batchSize) { - row = baseIterator.next() + while (rowIterator.hasNext && rowCount < batchSize) { + val row = rowIterator.next() var i = 0 while (i < row.length) { columnBuilders(i).appendFrom(row, i) @@ -80,7 +78,7 @@ private[sql] case class InMemoryRelation( CachedBatch(columnBuilders.map(_.build()), stats) } - def hasNext = baseIterator.hasNext + def hasNext = rowIterator.hasNext } }.cache() @@ -182,6 +180,7 @@ private[sql] case class InMemoryColumnarTableScan( } } + // Accumulators used for testing purposes val readPartitions = sparkContext.accumulator(0) val readBatches = sparkContext.accumulator(0) @@ -191,40 +190,36 @@ private[sql] case class InMemoryColumnarTableScan( readPartitions.setValue(0) readBatches.setValue(0) - relation.cachedColumnBuffers.mapPartitions { iterator => + relation.cachedColumnBuffers.mapPartitions { cachedBatchIterator => val partitionFilter = newPredicate( partitionFilters.reduceOption(And).getOrElse(Literal(true)), relation.partitionStatistics.schema) - // Find the ordinals of the requested columns. If none are requested, use the first. - val requestedColumns = if (attributes.isEmpty) { - Seq(0) + // Find the ordinals and data types of the requested columns. If none are requested, use the + // narrowest (the field with minimum default element size). + val (requestedColumnIndices, requestedColumnDataTypes) = if (attributes.isEmpty) { + val (narrowestOrdinal, narrowestDataType) = + relation.output.zipWithIndex.map { case (a, ordinal) => + ordinal -> a.dataType + } minBy { case (_, dataType) => + ColumnType(dataType).defaultSize + } + Seq(narrowestOrdinal) -> Seq(narrowestDataType) } else { - attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) + attributes.map { a => + relation.output.indexWhere(_.exprId == a.exprId) -> a.dataType + }.unzip } - val rows = iterator - // Skip pruned batches - .filter { cachedBatch => - if (inMemoryPartitionPruningEnabled && !partitionFilter(cachedBatch.stats)) { - def statsString = relation.partitionStatistics.schema - .zip(cachedBatch.stats) - .map { case (a, s) => s"${a.name}: $s" } - .mkString(", ") - logInfo(s"Skipping partition based on stats $statsString") - false - } else { - readBatches += 1 - true - } - } - // Build column accessors - .map { cachedBatch => - requestedColumns.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) - } - // Extract rows via column accessors - .flatMap { columnAccessors => - val nextRow = new GenericMutableRow(columnAccessors.length) + val nextRow = new SpecificMutableRow(requestedColumnDataTypes) + + def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]) = { + val rows = cacheBatches.flatMap { cachedBatch => + // Build column accessors + val columnAccessors = + requestedColumnIndices.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) + + // Extract rows via column accessors new Iterator[Row] { override def next() = { var i = 0 @@ -235,15 +230,38 @@ private[sql] case class InMemoryColumnarTableScan( nextRow } - override def hasNext = columnAccessors.head.hasNext + override def hasNext = columnAccessors(0).hasNext } } - if (rows.hasNext) { - readPartitions += 1 + if (rows.hasNext) { + readPartitions += 1 + } + + rows } - rows + // Do partition batch pruning if enabled + val cachedBatchesToScan = + if (inMemoryPartitionPruningEnabled) { + cachedBatchIterator.filter { cachedBatch => + if (!partitionFilter(cachedBatch.stats)) { + def statsString = relation.partitionStatistics.schema + .zip(cachedBatch.stats) + .map { case (a, s) => s"${a.name}: $s" } + .mkString(", ") + logInfo(s"Skipping partition based on stats $statsString") + false + } else { + readBatches += 1 + true + } + } + } else { + cachedBatchIterator + } + + cachedBatchesToRows(cachedBatchesToScan) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index b7f8826861a2c..965782a40031b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -29,7 +29,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { private var nextNullIndex: Int = _ private var pos: Int = 0 - abstract override protected def initialize() { + abstract override protected def initialize(): Unit = { nullsBuffer = underlyingBuffer.duplicate().order(ByteOrder.nativeOrder()) nullCount = nullsBuffer.getInt() nextNullIndex = if (nullCount > 0) nullsBuffer.getInt() else -1 @@ -39,7 +39,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { super.initialize() } - abstract override def extractTo(row: MutableRow, ordinal: Int) { + abstract override def extractTo(row: MutableRow, ordinal: Int): Unit = { if (pos == nextNullIndex) { seenNulls += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index a72970eef7aa4..f1f494ac26d0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -40,7 +40,11 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { protected var nullCount: Int = _ private var pos: Int = _ - abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { + abstract override def initialize( + initialSize: Int, + columnName: String, + useCompression: Boolean): Unit = { + nulls = ByteBuffer.allocate(1024) nulls.order(ByteOrder.nativeOrder()) pos = 0 @@ -48,7 +52,7 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { super.initialize(initialSize, columnName, useCompression) } - abstract override def appendFrom(row: Row, ordinal: Int) { + abstract override def appendFrom(row: Row, ordinal: Int): Unit = { columnStats.gatherStats(row, ordinal) if (row.isNullAt(ordinal)) { nulls = ColumnBuilder.ensureFreeSpace(nulls, 4) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index b4120a3d4368b..27ac5f4dbdbbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.columnar.compression -import java.nio.ByteBuffer - +import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} @@ -34,5 +33,7 @@ private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAcc abstract override def hasNext = super.hasNext || decoder.hasNext - override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() + override def extractSingle(row: MutableRow, ordinal: Int): Unit = { + decoder.next(row, ordinal) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index a5826bb033e41..628d9cec41d6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -48,12 +48,16 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] var compressionEncoders: Seq[Encoder[T]] = _ - abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { + abstract override def initialize( + initialSize: Int, + columnName: String, + useCompression: Boolean): Unit = { + compressionEncoders = if (useCompression) { - schemes.filter(_.supports(columnType)).map(_.encoder[T]) + schemes.filter(_.supports(columnType)).map(_.encoder[T](columnType)) } else { - Seq(PassThrough.encoder) + Seq(PassThrough.encoder(columnType)) } super.initialize(initialSize, columnName, useCompression) } @@ -62,17 +66,15 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] encoder.compressionRatio < 0.8 } - private def gatherCompressibilityStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - + private def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { var i = 0 while (i < compressionEncoders.length) { - compressionEncoders(i).gatherCompressibilityStats(field, columnType) + compressionEncoders(i).gatherCompressibilityStats(row, ordinal) i += 1 } } - abstract override def appendFrom(row: Row, ordinal: Int) { + abstract override def appendFrom(row: Row, ordinal: Int): Unit = { super.appendFrom(row, ordinal) if (!row.isNullAt(ordinal)) { gatherCompressibilityStats(row, ordinal) @@ -84,7 +86,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] val typeId = nonNullBuffer.getInt() val encoder: Encoder[T] = { val candidate = compressionEncoders.minBy(_.compressionRatio) - if (isWorthCompressing(candidate)) candidate else PassThrough.encoder + if (isWorthCompressing(candidate)) candidate else PassThrough.encoder(columnType) } // Header = column type ID + null count + null positions @@ -104,7 +106,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] .putInt(nullCount) .put(nulls) - logInfo(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") - encoder.compress(nonNullBuffer, compressedBuffer, columnType) + logDebug(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + encoder.compress(nonNullBuffer, compressedBuffer) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index 7797f75177893..acb06cb5376b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.columnar.compression -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} private[sql] trait Encoder[T <: NativeType] { - def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) {} + def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {} def compressedSize: Int @@ -33,17 +35,21 @@ private[sql] trait Encoder[T <: NativeType] { if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0 } - def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]): ByteBuffer + def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer } -private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType] +private[sql] trait Decoder[T <: NativeType] { + def next(row: MutableRow, ordinal: Int): Unit + + def hasNext: Boolean +} private[sql] trait CompressionScheme { def typeId: Int def supports(columnType: ColumnType[_, _]): Boolean - def encoder[T <: NativeType]: Encoder[T] + def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 8cf9ec74ca2de..29edcf17242c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -23,7 +23,8 @@ import scala.collection.mutable import scala.reflect.ClassTag import scala.reflect.runtime.universe.runtimeMirror -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ import org.apache.spark.util.Utils @@ -33,18 +34,20 @@ private[sql] case object PassThrough extends CompressionScheme { override def supports(columnType: ColumnType[_, _]) = true - override def encoder[T <: NativeType] = new this.Encoder[T] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + new this.Encoder[T](columnType) + } override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) } - class Encoder[T <: NativeType] extends compression.Encoder[T] { + class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { override def uncompressedSize = 0 override def compressedSize = 0 - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { + override def compress(from: ByteBuffer, to: ByteBuffer) = { // Writes compression type ID and copies raw contents to.putInt(PassThrough.typeId).put(from).rewind() to @@ -54,7 +57,9 @@ private[sql] case object PassThrough extends CompressionScheme { class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { - override def next() = columnType.extract(buffer) + override def next(row: MutableRow, ordinal: Int): Unit = { + columnType.extract(buffer, row, ordinal) + } override def hasNext = buffer.hasRemaining } @@ -63,7 +68,9 @@ private[sql] case object PassThrough extends CompressionScheme { private[sql] case object RunLengthEncoding extends CompressionScheme { override val typeId = 1 - override def encoder[T <: NativeType] = new this.Encoder[T] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + new this.Encoder[T](columnType) + } override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) @@ -74,24 +81,25 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder[T <: NativeType] extends compression.Encoder[T] { + class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { private var _uncompressedSize = 0 private var _compressedSize = 0 // Using `MutableRow` to store the last value to avoid boxing/unboxing cost. - private val lastValue = new GenericMutableRow(1) + private val lastValue = new SpecificMutableRow(Seq(columnType.dataType)) private var lastRun = 0 override def uncompressedSize = _uncompressedSize override def compressedSize = _compressedSize - override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { - val actualSize = columnType.actualSize(value) + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = columnType.getField(row, ordinal) + val actualSize = columnType.actualSize(row, ordinal) _uncompressedSize += actualSize if (lastValue.isNullAt(0)) { - columnType.setField(lastValue, 0, value) + columnType.copyField(row, ordinal, lastValue, 0) lastRun = 1 _compressedSize += actualSize + 4 } else { @@ -99,37 +107,40 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { lastRun += 1 } else { _compressedSize += actualSize + 4 - columnType.setField(lastValue, 0, value) + columnType.copyField(row, ordinal, lastValue, 0) lastRun = 1 } } } - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { + override def compress(from: ByteBuffer, to: ByteBuffer) = { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { - var currentValue = columnType.extract(from) + val currentValue = new SpecificMutableRow(Seq(columnType.dataType)) var currentRun = 1 + val value = new SpecificMutableRow(Seq(columnType.dataType)) + + columnType.extract(from, currentValue, 0) while (from.hasRemaining) { - val value = columnType.extract(from) + columnType.extract(from, value, 0) - if (value == currentValue) { + if (value.head == currentValue.head) { currentRun += 1 } else { // Writes current run - columnType.append(currentValue, to) + columnType.append(currentValue, 0, to) to.putInt(currentRun) // Resets current run - currentValue = value + columnType.copyField(value, 0, currentValue, 0) currentRun = 1 } } // Writes the last run - columnType.append(currentValue, to) + columnType.append(currentValue, 0, to) to.putInt(currentRun) } @@ -145,7 +156,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { private var valueCount = 0 private var currentValue: T#JvmType = _ - override def next() = { + override def next(row: MutableRow, ordinal: Int): Unit = { if (valueCount == run) { currentValue = columnType.extract(buffer) run = buffer.getInt() @@ -154,7 +165,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { valueCount += 1 } - currentValue + columnType.setField(row, ordinal, currentValue) } override def hasNext = valueCount < run || buffer.hasRemaining @@ -171,14 +182,16 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { new this.Decoder(buffer, columnType) } - override def encoder[T <: NativeType] = new this.Encoder[T] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + new this.Encoder[T](columnType) + } override def supports(columnType: ColumnType[_, _]) = columnType match { case INT | LONG | STRING => true case _ => false } - class Encoder[T <: NativeType] extends compression.Encoder[T] { + class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary // overflows. private var _uncompressedSize = 0 @@ -200,9 +213,11 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // to store dictionary element count. private var dictionarySize = 4 - override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = columnType.getField(row, ordinal) + if (!overflow) { - val actualSize = columnType.actualSize(value) + val actualSize = columnType.actualSize(row, ordinal) count += 1 _uncompressedSize += actualSize @@ -221,7 +236,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { + override def compress(from: ByteBuffer, to: ByteBuffer) = { if (overflow) { throw new IllegalStateException( "Dictionary encoding should not be used because of dictionary overflow.") @@ -264,7 +279,9 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def next() = dictionary(buffer.getShort()) + override def next(row: MutableRow, ordinal: Int): Unit = { + columnType.setField(row, ordinal, dictionary(buffer.getShort())) + } override def hasNext = buffer.hasRemaining } @@ -279,25 +296,20 @@ private[sql] case object BooleanBitSet extends CompressionScheme { new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + (new this.Encoder).asInstanceOf[compression.Encoder[T]] + } override def supports(columnType: ColumnType[_, _]) = columnType == BOOLEAN class Encoder extends compression.Encoder[BooleanType.type] { private var _uncompressedSize = 0 - override def gatherCompressibilityStats( - value: Boolean, - columnType: NativeColumnType[BooleanType.type]) { - + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { _uncompressedSize += BOOLEAN.defaultSize } - override def compress( - from: ByteBuffer, - to: ByteBuffer, - columnType: NativeColumnType[BooleanType.type]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer) = { to.putInt(BooleanBitSet.typeId) // Total element count (1 byte per Boolean value) .putInt(from.remaining) @@ -349,7 +361,7 @@ private[sql] case object BooleanBitSet extends CompressionScheme { private var visited: Int = 0 - override def next(): Boolean = { + override def next(row: MutableRow, ordinal: Int): Unit = { val bit = visited % BITS_PER_LONG visited += 1 @@ -357,123 +369,167 @@ private[sql] case object BooleanBitSet extends CompressionScheme { currentWord = buffer.getLong() } - ((currentWord >> bit) & 1) != 0 + row.setBoolean(ordinal, ((currentWord >> bit) & 1) != 0) } override def hasNext: Boolean = visited < count } } -private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends CompressionScheme { +private[sql] case object IntDelta extends CompressionScheme { + override def typeId: Int = 4 + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { - new this.Decoder(buffer, columnType.asInstanceOf[NativeColumnType[I]]) - .asInstanceOf[compression.Decoder[T]] + new Decoder(buffer, INT).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] - - /** - * Computes `delta = x - y`, returns `(true, delta)` if `delta` can fit into a single byte, or - * `(false, 0: Byte)` otherwise. - */ - protected def byteSizedDelta(x: I#JvmType, y: I#JvmType): (Boolean, Byte) + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + (new Encoder).asInstanceOf[compression.Encoder[T]] + } - /** - * Simply computes `x + delta` - */ - protected def addDelta(x: I#JvmType, delta: Byte): I#JvmType + override def supports(columnType: ColumnType[_, _]) = columnType == INT - class Encoder extends compression.Encoder[I] { - private var _compressedSize: Int = 0 + class Encoder extends compression.Encoder[IntegerType.type] { + protected var _compressedSize: Int = 0 + protected var _uncompressedSize: Int = 0 - private var _uncompressedSize: Int = 0 + override def compressedSize = _compressedSize + override def uncompressedSize = _uncompressedSize - private var prev: I#JvmType = _ + private var prevValue: Int = _ - private var initial = true + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = row.getInt(ordinal) + val delta = value - prevValue - override def gatherCompressibilityStats(value: I#JvmType, columnType: NativeColumnType[I]) { - _uncompressedSize += columnType.defaultSize + _compressedSize += 1 - if (initial) { - initial = false - _compressedSize += 1 + columnType.defaultSize - } else { - val (smallEnough, _) = byteSizedDelta(value, prev) - _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) + // If this is the first integer to be compressed, or the delta is out of byte range, then give + // up compressing this integer. + if (_uncompressedSize == 0 || delta <= Byte.MinValue || delta > Byte.MaxValue) { + _compressedSize += INT.defaultSize } - prev = value + _uncompressedSize += INT.defaultSize + prevValue = value } - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { to.putInt(typeId) if (from.hasRemaining) { - var prev = columnType.extract(from) + var prev = from.getInt() to.put(Byte.MinValue) - columnType.append(prev, to) + to.putInt(prev) while (from.hasRemaining) { - val current = columnType.extract(from) - val (smallEnough, delta) = byteSizedDelta(current, prev) + val current = from.getInt() + val delta = current - prev prev = current - if (smallEnough) { - to.put(delta) + if (Byte.MinValue < delta && delta <= Byte.MaxValue) { + to.put(delta.toByte) } else { to.put(Byte.MinValue) - columnType.append(current, to) + to.putInt(current) } } } - to.rewind() - to + to.rewind().asInstanceOf[ByteBuffer] } - - override def uncompressedSize = _uncompressedSize - - override def compressedSize = _compressedSize } - class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[I]) - extends compression.Decoder[I] { + class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[IntegerType.type]) + extends compression.Decoder[IntegerType.type] { + + private var prev: Int = _ - private var prev: I#JvmType = _ + override def hasNext: Boolean = buffer.hasRemaining - override def next() = { + override def next(row: MutableRow, ordinal: Int): Unit = { val delta = buffer.get() - prev = if (delta > Byte.MinValue) addDelta(prev, delta) else columnType.extract(buffer) - prev + prev = if (delta > Byte.MinValue) prev + delta else buffer.getInt() + row.setInt(ordinal, prev) } - - override def hasNext = buffer.hasRemaining } } -private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { - override val typeId = 4 +private[sql] case object LongDelta extends CompressionScheme { + override def typeId: Int = 5 - override def supports(columnType: ColumnType[_, _]) = columnType == INT + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new Decoder(buffer, LONG).asInstanceOf[compression.Decoder[T]] + } + + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + (new Encoder).asInstanceOf[compression.Encoder[T]] + } - override protected def addDelta(x: Int, delta: Byte) = x + delta + override def supports(columnType: ColumnType[_, _]) = columnType == LONG + + class Encoder extends compression.Encoder[LongType.type] { + protected var _compressedSize: Int = 0 + protected var _uncompressedSize: Int = 0 + + override def compressedSize = _compressedSize + override def uncompressedSize = _uncompressedSize + + private var prevValue: Long = _ + + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = row.getLong(ordinal) + val delta = value - prevValue + + _compressedSize += 1 - override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { - val delta = x - y - if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + // If this is the first long integer to be compressed, or the delta is out of byte range, then + // give up compressing this long integer. + if (_uncompressedSize == 0 || delta <= Byte.MinValue || delta > Byte.MaxValue) { + _compressedSize += LONG.defaultSize + } + + _uncompressedSize += LONG.defaultSize + prevValue = value + } + + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { + to.putInt(typeId) + + if (from.hasRemaining) { + var prev = from.getLong() + to.put(Byte.MinValue) + to.putLong(prev) + + while (from.hasRemaining) { + val current = from.getLong() + val delta = current - prev + prev = current + + if (Byte.MinValue < delta && delta <= Byte.MaxValue) { + to.put(delta.toByte) + } else { + to.put(Byte.MinValue) + to.putLong(current) + } + } + } + + to.rewind().asInstanceOf[ByteBuffer] + } } -} -private[sql] case object LongDelta extends IntegralDelta[LongType.type] { - override val typeId = 5 + class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[LongType.type]) + extends compression.Decoder[LongType.type] { - override def supports(columnType: ColumnType[_, _]) = columnType == LONG + private var prev: Long = _ - override protected def addDelta(x: Long, delta: Byte) = x + delta + override def hasNext: Boolean = buffer.hasRemaining - override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { - val delta = x - y - if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + override def next(row: MutableRow, ordinal: Int): Unit = { + val delta = buffer.get() + prev = if (delta > Byte.MinValue) prev + delta else buffer.getLong() + row.setLong(ordinal, prev) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index cde91ceb68c98..0cdbb3167ce36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -35,7 +35,7 @@ class ColumnStatsSuite extends FunSuite { def testColumnStats[T <: NativeType, U <: ColumnStats]( columnStatsClass: Class[U], columnType: NativeColumnType[T], - initialStatistics: Row) { + initialStatistics: Row): Unit = { val columnStatsName = columnStatsClass.getSimpleName diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 75f653f3280bd..4fb1ecf1d532b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -23,6 +23,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -46,10 +47,12 @@ class ColumnTypeSuite extends FunSuite with Logging { def checkActualSize[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], value: JvmType, - expected: Int) { + expected: Int): Unit = { assertResult(expected, s"Wrong actualSize for $columnType") { - columnType.actualSize(value) + val row = new GenericMutableRow(1) + columnType.setField(row, 0, value) + columnType.actualSize(row, 0) } } @@ -147,7 +150,7 @@ class ColumnTypeSuite extends FunSuite with Logging { def testNativeColumnType[T <: NativeType]( columnType: NativeColumnType[T], putter: (ByteBuffer, T#JvmType) => Unit, - getter: (ByteBuffer) => T#JvmType) { + getter: (ByteBuffer) => T#JvmType): Unit = { testColumnType[T, T#JvmType](columnType, putter, getter) } @@ -155,7 +158,7 @@ class ColumnTypeSuite extends FunSuite with Logging { def testColumnType[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], putter: (ByteBuffer, JvmType) => Unit, - getter: (ByteBuffer) => JvmType) { + getter: (ByteBuffer) => JvmType): Unit = { val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE) val seq = (0 until 4).map(_ => makeRandomValue(columnType)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 0e3c67f5eed29..c1278248ef655 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{SQLConf, QueryTest, TestData} +import org.apache.spark.sql.{QueryTest, TestData} class InMemoryColumnarQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 3baa6f8ec0c83..6c9a9ab6c3418 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -45,7 +45,9 @@ class NullableColumnAccessorSuite extends FunSuite { testNullableColumnAccessor(_) } - def testNullableColumnAccessor[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { + def testNullableColumnAccessor[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType]): Unit = { + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") val nullRow = makeNullRow(1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index a77262534a352..f54a21eb4fbb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -41,7 +41,9 @@ class NullableColumnBuilderSuite extends FunSuite { testNullableColumnBuilder(_) } - def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { + def testNullableColumnBuilder[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType]): Unit = { + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") test(s"$typeName column builder: empty column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 5d2fd4959197c..69e0adbd3ee0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -28,7 +28,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be val originalColumnBatchSize = columnBatchSize val originalInMemoryPartitionPruning = inMemoryPartitionPruning - override protected def beforeAll() { + override protected def beforeAll(): Unit = { // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch setConf(SQLConf.COLUMN_BATCH_SIZE, "10") val rawData = sparkContext.makeRDD(1 to 100, 5).map(IntegerData) @@ -38,7 +38,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") } - override protected def afterAll() { + override protected def afterAll(): Unit = { setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString) setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString) } @@ -76,7 +76,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be filter: String, expectedQueryResult: Seq[Int], expectedReadPartitions: Int, - expectedReadBatches: Int) { + expectedReadBatches: Int): Unit = { test(filter) { val query = sql(s"SELECT * FROM intData WHERE $filter") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index e01cc8b4d20f2..d9e488e0ffd16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar.{NoopColumnStats, BOOLEAN} import org.apache.spark.sql.columnar.ColumnarTestUtils._ @@ -72,10 +73,14 @@ class BooleanBitSetSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) + val mutableRow = new GenericMutableRow(1) if (values.nonEmpty) { values.foreach { assert(decoder.hasNext) - assertResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value") { + decoder.next(mutableRow, 0) + mutableRow.getBoolean(0) + } } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index d2969d906c943..1cdb909146d57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ @@ -67,7 +68,7 @@ class DictionaryEncodingSuite extends FunSuite { val buffer = builder.build() val headerSize = CompressionScheme.columnHeaderSize(buffer) // 4 extra bytes for dictionary size - val dictionarySize = 4 + values.map(columnType.actualSize).sum + val dictionarySize = 4 + rows.map(columnType.actualSize(_, 0)).sum // 2 bytes for each `Short` val compressedSize = 4 + dictionarySize + 2 * inputSeq.length // 4 extra bytes for compression scheme type ID @@ -97,11 +98,15 @@ class DictionaryEncodingSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = DictionaryEncoding.decoder(buffer, columnType) + val mutableRow = new GenericMutableRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - assertResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 322f447c24840..73f31c0233343 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -31,7 +31,7 @@ class IntegralDeltaSuite extends FunSuite { def testIntegralDelta[I <: IntegralType]( columnStats: ColumnStats, columnType: NativeColumnType[I], - scheme: IntegralDelta[I]) { + scheme: CompressionScheme) { def skeleton(input: Seq[I#JvmType]) { // ------------- @@ -96,10 +96,15 @@ class IntegralDeltaSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = scheme.decoder(buffer, columnType) + val mutableRow = new GenericMutableRow(1) + if (input.nonEmpty) { input.foreach{ assert(decoder.hasNext) - assertResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 218c09ac26362..4ce2552112c92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ @@ -57,7 +58,7 @@ class RunLengthEncodingSuite extends FunSuite { // Compression scheme ID + compressed contents val compressedSize = 4 + inputRuns.map { case (index, _) => // 4 extra bytes each run for run length - columnType.actualSize(values(index)) + 4 + columnType.actualSize(rows(index), 0) + 4 }.sum // 4 extra bytes for compression scheme type ID @@ -80,11 +81,15 @@ class RunLengthEncodingSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = RunLengthEncoding.decoder(buffer, columnType) + val mutableRow = new GenericMutableRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - assertResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 329f80cad471e..84fafcde63d05 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -25,16 +25,14 @@ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector - +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} - -import org.apache.spark.sql.catalyst.expressions.{Attribute, Row, GenericMutableRow, Literal, Cast} -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.expressions._ /** * A trait for subclasses that handle table scans. @@ -108,12 +106,12 @@ class HadoopTableReader( val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) val attrsWithIndex = attributes.zipWithIndex - val mutableRow = new GenericMutableRow(attrsWithIndex.length) + val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) - HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) } @@ -164,33 +162,32 @@ class HadoopTableReader( val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHiveConf val localDeserializer = partDeserializer - val mutableRow = new GenericMutableRow(attributes.length) - - // split the attributes (output schema) into 2 categories: - // (partition keys, ordinal), (normal attributes, ordinal), the ordinal mean the - // index of the attribute in the output Row. - val (partitionKeys, attrs) = attributes.zipWithIndex.partition(attr => { - relation.partitionKeys.indexOf(attr._1) >= 0 - }) - - def fillPartitionKeys(parts: Array[String], row: GenericMutableRow) = { - partitionKeys.foreach { case (attr, ordinal) => - // get partition key ordinal for a given attribute - val partOridinal = relation.partitionKeys.indexOf(attr) - row(ordinal) = Cast(Literal(parts(partOridinal)), attr.dataType).eval(null) + val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + + // Splits all attributes into two groups, partition key attributes and those that are not. + // Attached indices indicate the position of each attribute in the output schema. + val (partitionKeyAttrs, nonPartitionKeyAttrs) = + attributes.zipWithIndex.partition { case (attr, _) => + relation.partitionKeys.contains(attr) + } + + def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow) = { + partitionKeyAttrs.foreach { case (attr, ordinal) => + val partOrdinal = relation.partitionKeys.indexOf(attr) + row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) } } - // fill the partition key for the given MutableRow Object + + // Fill all partition keys to the given MutableRow object fillPartitionKeys(partValues, mutableRow) - val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) - hivePartitionRDD.mapPartitions { iter => + createHadoopRdd(tableDesc, inputPathStr, ifc).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() deserializer.initialize(hconf, partProps) - // fill the non partition key attributes - HadoopTableReader.fillObject(iter, deserializer, attrs, mutableRow) + // fill the non partition key attributes + HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, mutableRow) } }.toSeq @@ -257,38 +254,64 @@ private[hive] object HadoopTableReader extends HiveInspectors { } /** - * Transform the raw data(Writable object) into the Row object for an iterable input - * @param iter Iterable input which represented as Writable object - * @param deserializer Deserializer associated with the input writable object - * @param attrs Represents the row attribute names and its zero-based position in the MutableRow - * @param row reusable MutableRow object - * - * @return Iterable Row object that transformed from the given iterable input. + * Transform all given raw `Writable`s into `Row`s. + * + * @param iterator Iterator of all `Writable`s to be transformed + * @param deserializer The `Deserializer` associated with the input `Writable` + * @param nonPartitionKeyAttrs Attributes that should be filled together with their corresponding + * positions in the output schema + * @param mutableRow A reusable `MutableRow` that should be filled + * @return An `Iterator[Row]` transformed from `iterator` */ def fillObject( - iter: Iterator[Writable], + iterator: Iterator[Writable], deserializer: Deserializer, - attrs: Seq[(Attribute, Int)], - row: GenericMutableRow): Iterator[Row] = { + nonPartitionKeyAttrs: Seq[(Attribute, Int)], + mutableRow: MutableRow): Iterator[Row] = { + val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector] - // get the field references according to the attributes(output of the reader) required - val fieldRefs = attrs.map { case (attr, idx) => (soi.getStructFieldRef(attr.name), idx) } + val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => + soi.getStructFieldRef(attr.name) -> ordinal + }.unzip + + // Builds specific unwrappers ahead of time according to object inspector types to avoid pattern + // matching and branching costs per row. + val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { + _.getFieldObjectInspector match { + case oi: BooleanObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) + case oi: ByteObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) + case oi: ShortObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) + case oi: IntObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) + case oi: LongObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) + case oi: FloatObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) + case oi: DoubleObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + case oi => + (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrapData(value, oi) + } + } // Map each tuple to a row object - iter.map { value => + iterator.map { value => val raw = deserializer.deserialize(value) - var idx = 0; - while (idx < fieldRefs.length) { - val fieldRef = fieldRefs(idx)._1 - val fieldIdx = fieldRefs(idx)._2 - val fieldValue = soi.getStructFieldData(raw, fieldRef) - - row(fieldIdx) = unwrapData(fieldValue, fieldRef.getFieldObjectInspector()) - - idx += 1 + var i = 0 + while (i < fieldRefs.length) { + val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) + if (fieldValue == null) { + mutableRow.setNullAt(fieldOrdinals(i)) + } else { + unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) + } + i += 1 } - row: Row + mutableRow: Row } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6bf8d18a5c32c..8c8a8b124ac69 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -295,8 +295,16 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") test("implement identity function using case statement") { - val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src").collect().toSet - val expected = sql("SELECT key FROM src").collect().toSet + val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") + .map { case Row(i: Int) => i } + .collect() + .toSet + + val expected = sql("SELECT key FROM src") + .map { case Row(i: Int) => i } + .collect() + .toSet + assert(actual === expected) } @@ -559,9 +567,9 @@ class HiveQuerySuite extends HiveComparisonTest { val testVal = "test.val.0" val nonexistentKey = "nonexistent" val KV = "([^=]+)=([^=]*)".r - def collectResults(rdd: SchemaRDD): Set[(String, String)] = - rdd.collect().map { - case Row(key: String, value: String) => key -> value + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { + case Row(key: String, value: String) => key -> value case Row(KV(key, value)) => key -> value }.toSet clear() From 0f8c4edf4e750e3d11da27cc22c40b0489da7f37 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 13 Sep 2014 16:08:04 -0700 Subject: [PATCH 64/76] [SQL] Decrease partitions when testing Author: Michael Armbrust Closes #2164 from marmbrus/shufflePartitions and squashes the following commits: 0da1e8c [Michael Armbrust] test hax ef2d985 [Michael Armbrust] more test hacks. 2dabae3 [Michael Armbrust] more test fixes 0bdbf21 [Michael Armbrust] Make parquet tests less order dependent b42eeab [Michael Armbrust] increase test parallelism 80453d5 [Michael Armbrust] Decrease partitions when testing --- .../spark/sql/test/TestSQLContext.scala | 9 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 142 +++++------------- .../org/apache/spark/sql/hive/TestHive.scala | 7 +- 3 files changed, 51 insertions(+), 107 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index f2389f8f0591e..265b67737c475 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -18,8 +18,13 @@ package org.apache.spark.sql.test import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SQLConf, SQLContext} /** A SQLContext that can be used for local testing. */ object TestSQLContext - extends SQLContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + extends SQLContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) { + + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index b0a06cd3ca090..08f7358446b29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -58,8 +58,7 @@ case class AllDataTypes( doubleField: Double, shortField: Short, byteField: Byte, - booleanField: Boolean, - binaryField: Array[Byte]) + booleanField: Boolean) case class AllDataTypesWithNonPrimitiveType( stringField: String, @@ -70,13 +69,14 @@ case class AllDataTypesWithNonPrimitiveType( shortField: Short, byteField: Byte, booleanField: Boolean, - binaryField: Array[Byte], array: Seq[Int], arrayContainsNull: Seq[Option[Int]], map: Map[Int, Long], mapValueContainsNull: Map[Int, Option[Long]], data: Data) +case class BinaryData(binaryData: Array[Byte]) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -108,26 +108,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - TestSQLContext.sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, - (0 to x).map(_.toByte).toArray)) - .saveAsParquetFile(tempDir) - val result = parquetFile(tempDir).collect() - range.foreach { - i => - assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") - assert(result(i).getInt(1) === i) - assert(result(i).getLong(2) === i.toLong) - assert(result(i).getFloat(3) === i.toFloat) - assert(result(i).getDouble(4) === i.toDouble) - assert(result(i).getShort(5) === i.toShort) - assert(result(i).getByte(6) === i.toByte) - assert(result(i).getBoolean(7) === (i % 2 == 0)) - assert(result(i)(8) === (0 to i).map(_.toByte).toArray) - } + val data = sparkContext.parallelize(range) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + + data.saveAsParquetFile(tempDir) + + checkAnswer( + parquetFile(tempDir), + data.toSchemaRDD.collect().toSeq) } - test("Treat binary as string") { + test("read/write binary data") { + // Since equality for Array[Byte] is broken we test this separately. + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + sparkContext.parallelize(BinaryData("test".getBytes("utf8")) :: Nil).saveAsParquetFile(tempDir) + parquetFile(tempDir) + .map(r => new String(r(0).asInstanceOf[Array[Byte]], "utf8")) + .collect().toSeq == Seq("test") + } + + ignore("Treat binary as string") { val oldIsParquetBinaryAsString = TestSQLContext.isParquetBinaryAsString // Create the test file. @@ -142,37 +142,16 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA StructField("c2", BinaryType, false) :: Nil) val schemaRDD1 = applySchema(rowRDD, schema) schemaRDD1.saveAsParquetFile(path) - val resultWithBinary = parquetFile(path).collect - range.foreach { - i => - assert(resultWithBinary(i).getInt(0) === i) - assert(resultWithBinary(i)(1) === s"val_$i".getBytes) - } - - TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") - // This ParquetRelation always use Parquet types to derive output. - val parquetRelation = new ParquetRelation( - path.toString, - Some(TestSQLContext.sparkContext.hadoopConfiguration), - TestSQLContext) { - override val output = - ParquetTypesConverter.convertToAttributes( - ParquetTypesConverter.readMetaData(new Path(path), conf).getFileMetaData.getSchema, - TestSQLContext.isParquetBinaryAsString) - } - val schemaRDD = new SchemaRDD(TestSQLContext, parquetRelation) - val resultWithString = schemaRDD.collect - range.foreach { - i => - assert(resultWithString(i).getInt(0) === i) - assert(resultWithString(i)(1) === s"val_$i") - } + checkAnswer( + parquetFile(path).select('c1, 'c2.cast(StringType)), + schemaRDD1.select('c1, 'c2.cast(StringType)).collect().toSeq) - schemaRDD.registerTempTable("tmp") + setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") + parquetFile(path).printSchema() checkAnswer( - sql("SELECT c1, c2 FROM tmp WHERE c2 = 'val_5' OR c2 = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) + parquetFile(path), + schemaRDD1.select('c1, 'c2.cast(StringType)).collect().toSeq) + // Set it back. TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) @@ -275,34 +254,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - TestSQLContext.sparkContext.parallelize(range) + val data = sparkContext.parallelize(range) .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, - (0 to x).map(_.toByte).toArray, (0 until x), (0 until x).map(Option(_).filter(_ % 3 == 0)), (0 until x).map(i => i -> i.toLong).toMap, (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), Data((0 until x), Nested(x, s"$x")))) - .saveAsParquetFile(tempDir) - val result = parquetFile(tempDir).collect() - range.foreach { - i => - assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") - assert(result(i).getInt(1) === i) - assert(result(i).getLong(2) === i.toLong) - assert(result(i).getFloat(3) === i.toFloat) - assert(result(i).getDouble(4) === i.toDouble) - assert(result(i).getShort(5) === i.toShort) - assert(result(i).getByte(6) === i.toByte) - assert(result(i).getBoolean(7) === (i % 2 == 0)) - assert(result(i)(8) === (0 to i).map(_.toByte).toArray) - assert(result(i)(9) === (0 until i)) - assert(result(i)(10) === (0 until i).map(i => if (i % 3 == 0) i else null)) - assert(result(i)(11) === (0 until i).map(i => i -> i.toLong).toMap) - assert(result(i)(12) === (0 until i).map(i => i -> i.toLong).toMap + (i -> null)) - assert(result(i)(13) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) - } + data.saveAsParquetFile(tempDir) + + checkAnswer( + parquetFile(tempDir), + data.toSchemaRDD.collect().toSeq) } test("self-join parquet files") { @@ -399,23 +363,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } } - test("Saving case class RDD table to file and reading it back in") { - val file = getTempFilePath("parquet") - val path = file.toString - val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) - .map(i => TestRDDEntry(i, s"val_$i")) - rdd.saveAsParquetFile(path) - val readFile = parquetFile(path) - readFile.registerTempTable("tmpx") - val rdd_copy = sql("SELECT * FROM tmpx").collect() - val rdd_orig = rdd.collect() - for(i <- 0 to 99) { - assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i") - assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value error in line $i") - } - Utils.deleteRecursively(file) - } - test("Read a parquet file instead of a directory") { val file = getTempFilePath("parquet") val path = file.toString @@ -448,32 +395,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() assert(rdd_copy1.size === 100) - assert(rdd_copy1(0).apply(0) === 1) - assert(rdd_copy1(0).apply(1) === "val_1") - // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is - // executed twice otherwise?! + sql("INSERT INTO dest SELECT * FROM source") - val rdd_copy2 = sql("SELECT * FROM dest").collect() + val rdd_copy2 = sql("SELECT * FROM dest").collect().sortBy(_.getInt(0)) assert(rdd_copy2.size === 200) - assert(rdd_copy2(0).apply(0) === 1) - assert(rdd_copy2(0).apply(1) === "val_1") - assert(rdd_copy2(99).apply(0) === 100) - assert(rdd_copy2(99).apply(1) === "val_100") - assert(rdd_copy2(100).apply(0) === 1) - assert(rdd_copy2(100).apply(1) === "val_1") Utils.deleteRecursively(dirname) } test("Insert (appending) to same table via Scala API") { - // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is - // executed twice otherwise?! sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) assert(double_rdd.size === 30) - for(i <- (0 to 14)) { - assert(double_rdd(i) === double_rdd(i+15), s"error: lines $i and ${i+15} to not match") - } + // let's restore the original test data Utils.deleteRecursively(ParquetTestData.testDir) ParquetTestData.writeFile() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a3bfd3a8f1fd2..70fb15259e7d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -35,12 +35,13 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{CacheCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive._ +import org.apache.spark.sql.SQLConf /* Implicit conversions */ import scala.collection.JavaConversions._ object TestHive - extends TestHiveContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + extends TestHiveContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) /** * A locally running test instance of Spark's Hive execution engine. @@ -90,6 +91,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists From 2aea0da84c58a179917311290083456dfa043db7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 13 Sep 2014 16:22:04 -0700 Subject: [PATCH 65/76] [SPARK-3030] [PySpark] Reuse Python worker Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker --- .../scala/org/apache/spark/SparkEnv.scala | 8 ++ .../apache/spark/api/python/PythonRDD.scala | 58 ++++++++++--- .../api/python/PythonWorkerFactory.scala | 85 ++++++++++++++++--- docs/configuration.md | 10 +++ python/pyspark/daemon.py | 38 ++++----- python/pyspark/mllib/_common.py | 12 ++- python/pyspark/serializers.py | 4 + python/pyspark/tests.py | 35 ++++++++ python/pyspark/worker.py | 9 +- 9 files changed, 208 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index dd95e406f2a8e..009ed64775844 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -108,6 +108,14 @@ class SparkEnv ( pythonWorkers.get(key).foreach(_.stopWorker(worker)) } } + + private[spark] + def releasePythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) { + synchronized { + val key = (pythonExec, envVars) + pythonWorkers.get(key).foreach(_.releaseWorker(worker)) + } + } } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ae8010300a500..ca8eef5f99edf 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -23,6 +23,7 @@ import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ +import scala.collection.mutable import scala.language.existentials import scala.reflect.ClassTag import scala.util.{Try, Success, Failure} @@ -52,6 +53,7 @@ private[spark] class PythonRDD( extends RDD[Array[Byte]](parent) { val bufferSize = conf.getInt("spark.buffer.size", 65536) + val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true) override def getPartitions = parent.partitions @@ -63,19 +65,26 @@ private[spark] class PythonRDD( val localdir = env.blockManager.diskBlockManager.localDirs.map( f => f.getPath()).mkString(",") envVars += ("SPARK_LOCAL_DIRS" -> localdir) // it's also used in monitor thread + if (reuse_worker) { + envVars += ("SPARK_REUSE_WORKER" -> "1") + } val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator val writerThread = new WriterThread(env, worker, split, context) + var complete_cleanly = false context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() - - // Cleanup the worker socket. This will also cause the Python worker to exit. - try { - worker.close() - } catch { - case e: Exception => logWarning("Failed to close worker socket", e) + if (reuse_worker && complete_cleanly) { + env.releasePythonWorker(pythonExec, envVars.toMap, worker) + } else { + try { + worker.close() + } catch { + case e: Exception => + logWarning("Failed to close worker socket", e) + } } } @@ -133,6 +142,7 @@ private[spark] class PythonRDD( stream.readFully(update) accumulator += Collections.singletonList(update) } + complete_cleanly = true null } } catch { @@ -195,11 +205,26 @@ private[spark] class PythonRDD( PythonRDD.writeUTF(include, dataOut) } // Broadcast variables - dataOut.writeInt(broadcastVars.length) + val oldBids = PythonRDD.getWorkerBroadcasts(worker) + val newBids = broadcastVars.map(_.id).toSet + // number of different broadcasts + val cnt = oldBids.diff(newBids).size + newBids.diff(oldBids).size + dataOut.writeInt(cnt) + for (bid <- oldBids) { + if (!newBids.contains(bid)) { + // remove the broadcast from worker + dataOut.writeLong(- bid - 1) // bid >= 0 + oldBids.remove(bid) + } + } for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) + if (!oldBids.contains(broadcast.id)) { + // send new broadcast + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + oldBids.add(broadcast.id) + } } dataOut.flush() // Serialized command: @@ -207,17 +232,18 @@ private[spark] class PythonRDD( dataOut.write(command) // Data values PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) + dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) dataOut.flush() } catch { case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) + worker.shutdownOutput() case e: Exception => // We must avoid throwing exceptions here, because the thread uncaught exception handler // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e - } finally { - Try(worker.shutdownOutput()) // kill Python worker process + worker.shutdownOutput() } } } @@ -278,6 +304,14 @@ private object SpecialLengths { private[spark] object PythonRDD extends Logging { val UTF8 = Charset.forName("UTF-8") + // remember the broadcasts sent to each worker + private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]() + private def getWorkerBroadcasts(worker: Socket) = { + synchronized { + workerBroadcasts.getOrElseUpdate(worker, new mutable.HashSet[Long]()) + } + } + /** * Adapter for calling SparkContext#runJob from Python. * diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 4c4796f6c59ba..71bdf0fe1b917 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -40,7 +40,10 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 - var daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + val daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + val idleWorkers = new mutable.Queue[Socket]() + var lastActivity = 0L + new MonitorThread().start() var simpleWorkers = new mutable.WeakHashMap[Socket, Process]() @@ -51,6 +54,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String def create(): Socket = { if (useDaemon) { + synchronized { + if (idleWorkers.size > 0) { + return idleWorkers.dequeue() + } + } createThroughDaemon() } else { createSimpleWorker() @@ -199,9 +207,44 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } + /** + * Monitor all the idle workers, kill them after timeout. + */ + private class MonitorThread extends Thread(s"Idle Worker Monitor for $pythonExec") { + + setDaemon(true) + + override def run() { + while (true) { + synchronized { + if (lastActivity + IDLE_WORKER_TIMEOUT_MS < System.currentTimeMillis()) { + cleanupIdleWorkers() + lastActivity = System.currentTimeMillis() + } + } + Thread.sleep(10000) + } + } + } + + private def cleanupIdleWorkers() { + while (idleWorkers.length > 0) { + val worker = idleWorkers.dequeue() + try { + // the worker will exit after closing the socket + worker.close() + } catch { + case e: Exception => + logWarning("Failed to close worker socket", e) + } + } + } + private def stopDaemon() { synchronized { if (useDaemon) { + cleanupIdleWorkers() + // Request shutdown of existing daemon by sending SIGTERM if (daemon != null) { daemon.destroy() @@ -220,23 +263,43 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } def stopWorker(worker: Socket) { - if (useDaemon) { - if (daemon != null) { - daemonWorkers.get(worker).foreach { pid => - // tell daemon to kill worker by pid - val output = new DataOutputStream(daemon.getOutputStream) - output.writeInt(pid) - output.flush() - daemon.getOutputStream.flush() + synchronized { + if (useDaemon) { + if (daemon != null) { + daemonWorkers.get(worker).foreach { pid => + // tell daemon to kill worker by pid + val output = new DataOutputStream(daemon.getOutputStream) + output.writeInt(pid) + output.flush() + daemon.getOutputStream.flush() + } } + } else { + simpleWorkers.get(worker).foreach(_.destroy()) } - } else { - simpleWorkers.get(worker).foreach(_.destroy()) } worker.close() } + + def releaseWorker(worker: Socket) { + if (useDaemon) { + synchronized { + lastActivity = System.currentTimeMillis() + idleWorkers.enqueue(worker) + } + } else { + // Cleanup the worker socket. This will also cause the Python worker to exit. + try { + worker.close() + } catch { + case e: Exception => + logWarning("Failed to close worker socket", e) + } + } + } } private object PythonWorkerFactory { val PROCESS_WAIT_TIMEOUT_MS = 10000 + val IDLE_WORKER_TIMEOUT_MS = 60000 // kill idle workers after 1 minute } diff --git a/docs/configuration.md b/docs/configuration.md index 36178efb97103..af16489a44281 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -206,6 +206,16 @@ Apart from these, the following properties are also available, and may be useful used during aggregation goes above this amount, it will spill the data into disks. + + spark.python.worker.reuse + true + + Reuse Python worker or not. If yes, it will use a fixed number of Python workers, + does not need to fork() a Python process for every tasks. It will be very useful + if there is large broadcast, then the broadcast will not be needed to transfered + from JVM to Python worker for every task. + + spark.executorEnv.[EnvironmentVariableName] (none) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 15445abf67147..64d6202acb27d 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -23,6 +23,7 @@ import sys import traceback import time +import gc from errno import EINTR, ECHILD, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN @@ -46,17 +47,6 @@ def worker(sock): signal.signal(SIGCHLD, SIG_DFL) signal.signal(SIGTERM, SIG_DFL) - # Blocks until the socket is closed by draining the input stream - # until it raises an exception or returns EOF. - def waitSocketClose(sock): - try: - while True: - # Empty string is returned upon EOF (and only then). - if sock.recv(4096) == '': - return - except: - pass - # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because # otherwise writes also cause a seek that makes us miss data on the read side. @@ -64,17 +54,13 @@ def waitSocketClose(sock): outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) exit_code = 0 try: - # Acknowledge that the fork was successful - write_int(os.getpid(), outfile) - outfile.flush() worker_main(infile, outfile) except SystemExit as exc: - exit_code = exc.code + exit_code = compute_real_exit_code(exc.code) finally: outfile.flush() - # The Scala side will close the socket upon task completion. - waitSocketClose(sock) - os._exit(compute_real_exit_code(exit_code)) + if exit_code: + os._exit(exit_code) # Cleanup zombie children @@ -111,6 +97,8 @@ def handle_sigterm(*args): signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP + reuse = os.environ.get("SPARK_REUSE_WORKER") + # Initialization complete try: while True: @@ -163,7 +151,19 @@ def handle_sigterm(*args): # in child process listen_sock.close() try: - worker(sock) + # Acknowledge that the fork was successful + outfile = sock.makefile("w") + write_int(os.getpid(), outfile) + outfile.flush() + outfile.close() + while True: + worker(sock) + if not reuse: + # wait for closing + while sock.recv(1024): + pass + break + gc.collect() except: traceback.print_exc() os._exit(1) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index bb60d3d0c8463..68f6033616726 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -21,7 +21,7 @@ from numpy import ndarray, float64, int64, int32, array_equal, array from pyspark import SparkContext, RDD from pyspark.mllib.linalg import SparseVector -from pyspark.serializers import Serializer +from pyspark.serializers import FramedSerializer """ @@ -451,18 +451,16 @@ def _serialize_rating(r): return ba -class RatingDeserializer(Serializer): +class RatingDeserializer(FramedSerializer): - def loads(self, stream): - length = struct.unpack("!i", stream.read(4))[0] - ba = stream.read(length) - res = ndarray(shape=(3, ), buffer=ba, dtype=float64, offset=4) + def loads(self, string): + res = ndarray(shape=(3, ), buffer=string, dtype=float64, offset=4) return int(res[0]), int(res[1]), res[2] def load_stream(self, stream): while True: try: - yield self.loads(stream) + yield self._read_with_length(stream) except struct.error: return except EOFError: diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index a5f9341e819a9..ec3c6f055441d 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -144,6 +144,8 @@ def _write_with_length(self, obj, stream): def _read_with_length(self, stream): length = read_int(stream) + if length == SpecialLengths.END_OF_DATA_SECTION: + raise EOFError obj = stream.read(length) if obj == "": raise EOFError @@ -438,6 +440,8 @@ def __init__(self, use_unicode=False): def loads(self, stream): length = read_int(stream) + if length == SpecialLengths.END_OF_DATA_SECTION: + raise EOFError s = stream.read(length) return s.decode("utf-8") if self.use_unicode else s diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b687d695b01c4..747cd1767de7b 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1222,11 +1222,46 @@ def run(): except OSError: self.fail("daemon had been killed") + # run a normal job + rdd = self.sc.parallelize(range(100), 1) + self.assertEqual(100, rdd.map(str).count()) + def test_fd_leak(self): N = 1100 # fd limit is 1024 by default rdd = self.sc.parallelize(range(N), N) self.assertEquals(N, rdd.count()) + def test_after_exception(self): + def raise_exception(_): + raise Exception() + rdd = self.sc.parallelize(range(100), 1) + self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) + self.assertEqual(100, rdd.map(str).count()) + + def test_after_jvm_exception(self): + tempFile = tempfile.NamedTemporaryFile(delete=False) + tempFile.write("Hello World!") + tempFile.close() + data = self.sc.textFile(tempFile.name, 1) + filtered_data = data.filter(lambda x: True) + self.assertEqual(1, filtered_data.count()) + os.unlink(tempFile.name) + self.assertRaises(Exception, lambda: filtered_data.count()) + + rdd = self.sc.parallelize(range(100), 1) + self.assertEqual(100, rdd.map(str).count()) + + def test_accumulator_when_reuse_worker(self): + from pyspark.accumulators import INT_ACCUMULATOR_PARAM + acc1 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM) + self.sc.parallelize(range(100), 20).foreach(lambda x: acc1.add(x)) + self.assertEqual(sum(range(100)), acc1.value) + + acc2 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM) + self.sc.parallelize(range(100), 20).foreach(lambda x: acc2.add(x)) + self.assertEqual(sum(range(100)), acc2.value) + self.assertEqual(sum(range(100)), acc1.value) + class TestSparkSubmit(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 6805063e06798..61b8a74d060e8 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -69,9 +69,14 @@ def main(infile, outfile): ser = CompressedSerializer(pickleSer) for _ in range(num_broadcast_variables): bid = read_long(infile) - value = ser._read_with_length(infile) - _broadcastRegistry[bid] = Broadcast(bid, value) + if bid >= 0: + value = ser._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, value) + else: + bid = - bid - 1 + _broadcastRegistry.remove(bid) + _accumulatorRegistry.clear() command = pickleSer._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() From 4e3fbe8cdb6c6291e219195abb272f3c81f0ed63 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 13 Sep 2014 22:31:21 -0700 Subject: [PATCH 66/76] [SPARK-3463] [PySpark] aggregate and show spilled bytes in Python Aggregate the number of bytes spilled into disks during aggregation or sorting, show them in Web UI. ![spilled](https://cloud.githubusercontent.com/assets/40902/4209758/4b995562-386d-11e4-97c1-8e838ee1d4e3.png) This patch is blocked by SPARK-3465. (It includes a fix for that). Author: Davies Liu Closes #2336 from davies/metrics and squashes the following commits: e37df38 [Davies Liu] remove outdated comments 1245eb7 [Davies Liu] remove the temporary fix ebd2f43 [Davies Liu] Merge branch 'master' into metrics 7e4ad04 [Davies Liu] Merge branch 'master' into metrics fbe9029 [Davies Liu] show spilled bytes in Python in web ui --- .../apache/spark/api/python/PythonRDD.scala | 4 ++++ python/pyspark/shuffle.py | 19 ++++++++++++++++--- python/pyspark/tests.py | 15 ++++++++------- python/pyspark/worker.py | 14 ++++++++++---- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ca8eef5f99edf..d5002fa02992b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -124,6 +124,10 @@ private[spark] class PythonRDD( val total = finishTime - startTime logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) + val memoryBytesSpilled = stream.readLong() + val diskBytesSpilled = stream.readLong() + context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += diskBytesSpilled read() case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 49829f5280a5f..ce597cbe91e15 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -68,6 +68,11 @@ def _get_local_dirs(sub): return [os.path.join(d, "python", str(os.getpid()), sub) for d in dirs] +# global stats +MemoryBytesSpilled = 0L +DiskBytesSpilled = 0L + + class Aggregator(object): """ @@ -313,10 +318,12 @@ def _spill(self): It will dump the data in batch for better performance. """ + global MemoryBytesSpilled, DiskBytesSpilled path = self._get_spill_dir(self.spills) if not os.path.exists(path): os.makedirs(path) + used_memory = get_used_memory() if not self.pdata: # The data has not been partitioned, it will iterator the # dataset once, write them into different files, has no @@ -334,6 +341,7 @@ def _spill(self): self.serializer.dump_stream([(k, v)], streams[h]) for s in streams: + DiskBytesSpilled += s.tell() s.close() self.data.clear() @@ -346,9 +354,11 @@ def _spill(self): # dump items in batch self.serializer.dump_stream(self.pdata[i].iteritems(), f) self.pdata[i].clear() + DiskBytesSpilled += os.path.getsize(p) self.spills += 1 gc.collect() # release the memory as much as possible + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 def iteritems(self): """ Return all merged items as iterator """ @@ -462,7 +472,6 @@ def __init__(self, memory_limit, serializer=None): self.memory_limit = memory_limit self.local_dirs = _get_local_dirs("sort") self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) - self._spilled_bytes = 0 def _get_path(self, n): """ Choose one directory for spill by number n """ @@ -476,6 +485,7 @@ def sorted(self, iterator, key=None, reverse=False): Sort the elements in iterator, do external sort when the memory goes above the limit. """ + global MemoryBytesSpilled, DiskBytesSpilled batch = 10 chunks, current_chunk = [], [] iterator = iter(iterator) @@ -486,15 +496,18 @@ def sorted(self, iterator, key=None, reverse=False): if len(chunk) < batch: break - if get_used_memory() > self.memory_limit: + used_memory = get_used_memory() + if used_memory > self.memory_limit: # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) with open(path, 'w') as f: self.serializer.dump_stream(current_chunk, f) - self._spilled_bytes += os.path.getsize(path) chunks.append(self.serializer.load_stream(open(path))) current_chunk = [] + gc.collect() + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + DiskBytesSpilled += os.path.getsize(path) elif not chunks: batch = min(batch * 2, 10000) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 747cd1767de7b..f3309a20fcffb 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -46,6 +46,7 @@ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType +from pyspark import shuffle _have_scipy = False _have_numpy = False @@ -138,17 +139,17 @@ def test_external_sort(self): random.shuffle(l) sorter = ExternalSorter(1) self.assertEquals(sorted(l), list(sorter.sorted(l))) - self.assertGreater(sorter._spilled_bytes, 0) - last = sorter._spilled_bytes + self.assertGreater(shuffle.DiskBytesSpilled, 0) + last = shuffle.DiskBytesSpilled self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) - self.assertGreater(sorter._spilled_bytes, last) - last = sorter._spilled_bytes + self.assertGreater(shuffle.DiskBytesSpilled, last) + last = shuffle.DiskBytesSpilled self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) - self.assertGreater(sorter._spilled_bytes, last) - last = sorter._spilled_bytes + self.assertGreater(shuffle.DiskBytesSpilled, last) + last = shuffle.DiskBytesSpilled self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), list(sorter.sorted(l, key=lambda x: -x, reverse=True))) - self.assertGreater(sorter._spilled_bytes, last) + self.assertGreater(shuffle.DiskBytesSpilled, last) def test_external_sort_in_rdd(self): conf = SparkConf().set("spark.python.worker.memory", "1m") diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 61b8a74d060e8..252176ac65fec 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,16 +23,14 @@ import time import socket import traceback -# CloudPickler needs to be imported so that depicklers are registered using the -# copy_reg module. + from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry -from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ CompressedSerializer - +from pyspark import shuffle pickleSer = PickleSerializer() utf8_deserializer = UTF8Deserializer() @@ -52,6 +50,11 @@ def main(infile, outfile): if split_index == -1: # for unit tests return + # initialize global state + shuffle.MemoryBytesSpilled = 0 + shuffle.DiskBytesSpilled = 0 + _accumulatorRegistry.clear() + # fetch name of workdir spark_files_dir = utf8_deserializer.loads(infile) SparkFiles._root_directory = spark_files_dir @@ -97,6 +100,9 @@ def main(infile, outfile): exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) + write_long(shuffle.MemoryBytesSpilled, outfile) + write_long(shuffle.DiskBytesSpilled, outfile) + # Mark the beginning of the accumulators section of the output write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) write_int(len(_accumulatorRegistry), outfile) From c243b21a8ba2610266702e00d7d4b5443cb1f687 Mon Sep 17 00:00:00 2001 From: Bertrand Bossy Date: Sun, 14 Sep 2014 21:10:17 -0700 Subject: [PATCH 67/76] SPARK-3039: Allow spark to be built using avro-mapred for hadoop2 SPARK-3039: Adds the maven property "avro.mapred.classifier" to build spark-assembly with avro-mapred with support for the new Hadoop API. Sets this property to hadoop2 for Hadoop 2 profiles. I am not very familiar with maven, nor do I know whether this potentially breaks something in the hive part of spark. There might be a more elegant way of doing this. Author: Bertrand Bossy Closes #1945 from bbossy/SPARK-3039 and squashes the following commits: c32ce59 [Bertrand Bossy] SPARK-3039: Allow spark to be built using avro-mapred for hadoop2 --- pom.xml | 5 +++++ sql/hive/pom.xml | 9 +++++++++ 2 files changed, 14 insertions(+) diff --git a/pom.xml b/pom.xml index 28763476f8313..520aed3806937 100644 --- a/pom.xml +++ b/pom.xml @@ -134,6 +134,7 @@ 0.3.6 3.0.0 1.7.6 + 0.7.1 1.8.3 1.1.0 @@ -621,6 +622,7 @@ org.apache.avro avro-mapred ${avro.version} + ${avro.mapred.classifier} io.netty @@ -1108,6 +1110,7 @@ 2.2.0 2.5.0 + hadoop2 @@ -1117,6 +1120,7 @@ 2.3.0 2.5.0 0.9.0 + hadoop2 @@ -1126,6 +1130,7 @@ 2.4.0 2.5.0 0.9.0 + hadoop2 diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 45a4c6dc98da0..9d7a02bf7b0b7 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -95,6 +95,15 @@ org.apache.avro avro + ${avro.version} + + + + org.apache.avro + avro-mapred + ${avro.version} + ${avro.mapred.classifier} org.scalatest From f493f7982b50e3c99e78b649e7c6c5b4313c5ffa Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 14 Sep 2014 21:17:29 -0700 Subject: [PATCH 68/76] [SPARK-3452] Maven build should skip publishing artifacts people shouldn... ...'t depend on Publish local in maven term is `install` and publish otherwise is `deploy` So disabled both for following projects. Author: Prashant Sharma Closes #2329 from ScrapCodes/SPARK-3452/maven-skip-install and squashes the following commits: 257b79a [Prashant Sharma] [SPARK-3452] Maven build should skip publishing artifacts people shouldn't depend on --- assembly/pom.xml | 14 ++++++++++++++ examples/pom.xml | 14 ++++++++++++++ extras/java8-tests/pom.xml | 14 ++++++++++++++ repl/pom.xml | 14 ++++++++++++++ tools/pom.xml | 14 ++++++++++++++ yarn/pom.xml | 14 ++++++++++++++ 6 files changed, 84 insertions(+) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4146168fc804b..604b1ab3de6a8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -88,6 +88,20 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins diff --git a/examples/pom.xml b/examples/pom.xml index 3f46c40464d3b..2b561857f9f33 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -203,6 +203,20 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-shade-plugin diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 8658ecf5abfab..7e478bed62da7 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -74,6 +74,20 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-surefire-plugin diff --git a/repl/pom.xml b/repl/pom.xml index fcc5f90d870e8..af528c8914335 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -99,6 +99,20 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.scalatest scalatest-maven-plugin diff --git a/tools/pom.xml b/tools/pom.xml index f36674476770c..b90eb0ca250c5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -63,6 +63,20 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-source-plugin diff --git a/yarn/pom.xml b/yarn/pom.xml index 7fcd7ee0d4547..815a736c2e8fd 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -88,6 +88,20 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.codehaus.mojo build-helper-maven-plugin From cc14644460872efb344e8d895859d70213a40840 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 15 Sep 2014 08:53:58 -0500 Subject: [PATCH 69/76] [SPARK-3410] The priority of shutdownhook for ApplicationMaster should not be integer literal I think, it need to keep the priority of shutdown hook for ApplicationMaster than the priority of shutdown hook for o.a.h.FileSystem depending on changing the priority for FileSystem. Author: Kousuke Saruta Closes #2283 from sarutak/SPARK-3410 and squashes the following commits: 1d44fef [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3410 bd6cc53 [Kousuke Saruta] Modified style ee6f1aa [Kousuke Saruta] Added constant "SHUTDOWN_HOOK_PRIORITY" to ApplicationMaster 54eb68f [Kousuke Saruta] Changed Shutdown hook priority to 20 2f0aee3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3410 4c5cb93 [Kousuke Saruta] Modified the priority for AM's shutdown hook 217d1a4 [Kousuke Saruta] Removed unused import statements 717aba2 [Kousuke Saruta] Modified ApplicationMaster to make to keep the priority of shutdown hook for ApplicationMaster higher than the priority of shutdown hook for HDFS --- .../spark/deploy/yarn/ApplicationMaster.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 735d7723b0ce6..cde5fff637a39 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -21,12 +21,8 @@ import java.io.IOException import java.net.Socket import java.util.concurrent.atomic.AtomicReference -import scala.collection.JavaConversions._ -import scala.util.Try - import akka.actor._ import akka.remote._ -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ @@ -107,8 +103,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } } - // Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using. - ShutdownHookManager.get().addShutdownHook(cleanupHook, 30) + + // Use higher priority than FileSystem. + assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) + ShutdownHookManager + .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserClass which does a @@ -407,6 +406,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, object ApplicationMaster extends Logging { + val SHUTDOWN_HOOK_PRIORITY: Int = 30 + private var master: ApplicationMaster = _ def main(args: Array[String]) = { From fe2b1d6a209db9fe96b1c6630677955b94bd48c9 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Mon, 15 Sep 2014 10:57:53 -0700 Subject: [PATCH 70/76] [SPARK-3425] do not set MaxPermSize for OpenJDK 1.8 Closes #2387 Author: Matthew Farrellee Closes #2301 from mattf/SPARK-3425 and squashes the following commits: 20f3c09 [Matthew Farrellee] [SPARK-3425] do not set MaxPermSize for OpenJDK 1.8 --- bin/spark-class | 2 +- dev/merge_spark_pr.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 5f5f9ea74888d..613dc9c4566f2 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -105,7 +105,7 @@ else exit 1 fi fi -JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size if [ "$JAVA_VERSION" -ge 18 ]; then diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index d48c8bde12905..a8e92e36fe0d8 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -44,9 +44,9 @@ # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") # ASF JIRA username -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") # ASF JIRA password -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "35500") GITHUB_BASE = "https://github.com/apache/spark/pull" GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" From e59fac1f97c3fbeeb6defd12625a49763a353156 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 15 Sep 2014 16:11:41 -0700 Subject: [PATCH 71/76] [SPARK-3518] Remove wasted statement in JsonProtocol Author: Kousuke Saruta Closes #2380 from sarutak/SPARK-3518 and squashes the following commits: 8a1464e [Kousuke Saruta] Replaced a variable with simple field reference c660fbc [Kousuke Saruta] Removed useless statement in JsonProtocol.scala --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b0754e3ce10db..c4dddb2d1037e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -205,7 +205,6 @@ private[spark] object JsonProtocol { } def taskInfoToJson(taskInfo: TaskInfo): JValue = { - val accumUpdateMap = taskInfo.accumulables ("Task ID" -> taskInfo.taskId) ~ ("Index" -> taskInfo.index) ~ ("Attempt" -> taskInfo.attempt) ~ From 37d925280cdfdda8f6f7174c67a614056eea5d69 Mon Sep 17 00:00:00 2001 From: yantangzhai Date: Mon, 15 Sep 2014 16:57:38 -0700 Subject: [PATCH 72/76] [SPARK-2714] DAGScheduler logs jobid when runJob finishes DAGScheduler logs jobid when runJob finishes Author: yantangzhai Closes #1617 from YanTangZhai/SPARK-2714 and squashes the following commits: 0a0243f [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes fbb1150 [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes 7aec2a9 [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes fb42f0f [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes 090d908 [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 9 +++++++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 218b353dd9d49..428f019b02a23 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1072,11 +1072,8 @@ class SparkContext(config: SparkConf) extends Logging { val callSite = getCallSite val cleanedFunc = clean(func) logInfo("Starting job: " + callSite.shortForm) - val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) - logInfo( - "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6fcf9e31543ed..b2774dfc47553 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -507,11 +507,16 @@ class DAGScheduler( resultHandler: (Int, U) => Unit, properties: Properties = null) { + val start = System.nanoTime val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { - case JobSucceeded => {} + case JobSucceeded => { + logInfo("Job %d finished: %s, took %f s".format + (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) + } case JobFailed(exception: Exception) => - logInfo("Failed to run " + callSite.shortForm) + logInfo("Job %d failed: %s, took %f s".format + (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) throw exception } } From 3b93128139e8d303f1d7bfd04e9a99a11a5b6404 Mon Sep 17 00:00:00 2001 From: Christoph Sawade Date: Mon, 15 Sep 2014 17:39:31 -0700 Subject: [PATCH 73/76] [SPARK-3396][MLLIB] Use SquaredL2Updater in LogisticRegressionWithSGD SimpleUpdater ignores the regularizer, which leads to an unregularized LogReg. To enable the common L2 regularizer (and the corresponding regularization parameter) for logistic regression the SquaredL2Updater has to be used in SGD (see, e.g., [SVMWithSGD]) Author: Christoph Sawade Closes #2398 from BigCrunsh/fix-regparam-logreg and squashes the following commits: 0820c04 [Christoph Sawade] Use SquaredL2Updater in LogisticRegressionWithSGD --- .../classification/LogisticRegression.scala | 2 +- .../LogisticRegressionSuite.scala | 44 +++++++++++++++++-- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 486bdbfa9cb47..84d3c7cebd7c8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -84,7 +84,7 @@ class LogisticRegressionWithSGD private ( extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { private val gradient = new LogisticGradient() - private val updater = new SimpleUpdater() + private val updater = new SquaredL2Updater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 862178694a50e..e954baaf7d91e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -43,7 +43,7 @@ object LogisticRegressionSuite { offset: Double, scale: Double, nPoints: Int, - seed: Int): Seq[LabeledPoint] = { + seed: Int): Seq[LabeledPoint] = { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) @@ -58,12 +58,15 @@ object LogisticRegressionSuite { } class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Matchers { - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction( + predictions: Seq[Double], + input: Seq[LabeledPoint], + expectedAcc: Double = 0.83) { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label } // At least 83% of the predictions should be on. - ((input.length - numOffPredictions).toDouble / input.length) should be > 0.83 + ((input.length - numOffPredictions).toDouble / input.length) should be > expectedAcc } // Test if we can correctly learn A, B where Y = logistic(A + B*X) @@ -155,6 +158,41 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + test("logistic regression with initial weights and non-default regularization parameter") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Vectors.dense(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegressionWithSGD().setIntercept(true) + lr.optimizer. + setStepSize(10.0). + setNumIterations(10). + setRegParam(1.0) + + val model = lr.run(testRDD, initialWeights) + + // Test the weights + assert(model.weights(0) ~== -430000.0 relTol 20000.0) + assert(model.intercept ~== 370000.0 relTol 20000.0) + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData, 0.8) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData, 0.8) + } + test("logistic regression with initial weights with LBFGS") { val nPoints = 10000 val A = 2.0 From 983d6a9c48b69c5f0542922aa8b133f69eb1034d Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Mon, 15 Sep 2014 17:41:15 -0700 Subject: [PATCH 74/76] [MLlib] Update SVD documentation in IndexedRowMatrix Updating this to reflect the newest SVD via ARPACK Author: Reza Zadeh Closes #2389 from rezazadeh/irmdocs and squashes the following commits: 7fa1313 [Reza Zadeh] Update svd docs 715da25 [Reza Zadeh] Updated computeSVD documentation IndexedRowMatrix --- .../mllib/linalg/distributed/IndexedRowMatrix.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index ac6eaea3f43ad..5c1acca0ec532 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -76,16 +76,12 @@ class IndexedRowMatrix( } /** - * Computes the singular value decomposition of this matrix. + * Computes the singular value decomposition of this IndexedRowMatrix. * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. * - * There is no restriction on m, but we require `n^2` doubles to fit in memory. - * Further, n should be less than m. - - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), from which we recover S and V. - * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). - * Note that this approach requires `O(n^3)` time on the master node. + * The cost and implementation of this method is identical to that in + * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] + * With the addition of indices. * * At most k largest non-zero singular values and associated vectors are returned. * If there are k such values, then the dimensions of the return will be: From fdb302f49c021227026909bdcdade7496059013f Mon Sep 17 00:00:00 2001 From: "qiping.lqp" Date: Mon, 15 Sep 2014 17:43:26 -0700 Subject: [PATCH 75/76] [SPARK-3516] [mllib] DecisionTree: Add minInstancesPerNode, minInfoGain params to example and Python API Added minInstancesPerNode, minInfoGain params to: * DecisionTreeRunner.scala example * Python API (tree.py) Also: * Fixed typo in tree suite test "do not choose split that does not satisfy min instance per node requirements" * small style fixes CC: mengxr Author: qiping.lqp Author: Joseph K. Bradley Author: chouqin Closes #2349 from jkbradley/chouqin-dt-preprune and squashes the following commits: 61b2e72 [Joseph K. Bradley] Added max of 10GB for maxMemoryInMB in Strategy. a95e7c8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune 95c479d [Joseph K. Bradley] * Fixed typo in tree suite test "do not choose split that does not satisfy min instance per node requirements" * small style fixes e2628b6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune 19b01af [Joseph K. Bradley] Merge remote-tracking branch 'chouqin/dt-preprune' into chouqin-dt-preprune f1d11d1 [chouqin] fix typo c7ebaf1 [chouqin] fix typo 39f9b60 [chouqin] change edge `minInstancesPerNode` to 2 and add one more test c6e2dfc [Joseph K. Bradley] Added minInstancesPerNode and minInfoGain parameters to DecisionTreeRunner.scala and to Python API in tree.py 0278a11 [chouqin] remove `noSplit` and set `Predict` private to tree d593ec7 [chouqin] fix docs and change minInstancesPerNode to 1 efcc736 [qiping.lqp] fix bug 10b8012 [qiping.lqp] fix style 6728fad [qiping.lqp] minor fix: remove empty lines bb465ca [qiping.lqp] Merge branch 'master' of https://github.com/apache/spark into dt-preprune cadd569 [qiping.lqp] add api docs 46b891f [qiping.lqp] fix bug e72c7e4 [qiping.lqp] add comments 845c6fa [qiping.lqp] fix style f195e83 [qiping.lqp] fix style 987cbf4 [qiping.lqp] fix bug ff34845 [qiping.lqp] separate calculation of predict of node from calculation of info gain ac42378 [qiping.lqp] add min info gain and min instances per node parameters in decision tree --- .../examples/mllib/DecisionTreeRunner.scala | 13 ++++++++++++- .../spark/mllib/api/python/PythonMLLibAPI.scala | 8 ++++++-- .../apache/spark/mllib/tree/DecisionTree.scala | 4 ++-- .../mllib/tree/configuration/Strategy.scala | 2 ++ .../apache/spark/mllib/tree/model/Predict.scala | 6 +----- .../spark/mllib/tree/DecisionTreeSuite.scala | 4 ++-- python/pyspark/mllib/tree.py | 16 ++++++++++++---- 7 files changed, 37 insertions(+), 16 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 72c3ab475b61f..4683e6eb966be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -55,6 +55,8 @@ object DecisionTreeRunner { maxDepth: Int = 5, impurity: ImpurityType = Gini, maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, fracTest: Double = 0.2) def main(args: Array[String]) { @@ -75,6 +77,13 @@ object DecisionTreeRunner { opt[Int]("maxBins") .text(s"max number of bins, default: ${defaultParams.maxBins}") .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") + .action((x, c) => c.copy(minInfoGain = x)) opt[Double]("fracTest") .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) @@ -179,7 +188,9 @@ object DecisionTreeRunner { impurity = impurityCalculator, maxDepth = params.maxDepth, maxBins = params.maxBins, - numClassesForClassification = numClasses) + numClassesForClassification = numClasses, + minInstancesPerNode = params.minInstancesPerNode, + minInfoGain = params.minInfoGain) val model = DecisionTree.train(training, strategy) println(model) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 4343124f102a0..fa0fa69f38634 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -303,7 +303,9 @@ class PythonMLLibAPI extends Serializable { categoricalFeaturesInfoJMap: java.util.Map[Int, Int], impurityStr: String, maxDepth: Int, - maxBins: Int): DecisionTreeModel = { + maxBins: Int, + minInstancesPerNode: Int, + minInfoGain: Double): DecisionTreeModel = { val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) @@ -316,7 +318,9 @@ class PythonMLLibAPI extends Serializable { maxDepth = maxDepth, numClassesForClassification = numClasses, maxBins = maxBins, - categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap) + categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap, + minInstancesPerNode = minInstancesPerNode, + minInfoGain = minInfoGain) DecisionTree.train(data, strategy) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 56bb8812100a7..c7f2576c822b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -389,7 +389,7 @@ object DecisionTree extends Serializable with Logging { var groupIndex = 0 var doneTraining = true while (groupIndex < numGroups) { - val (tmpRoot, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, + val (_, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, topNode, splits, bins, timer, numGroups, groupIndex) doneTraining = doneTraining && doneTrainingGroup groupIndex += 1 @@ -898,7 +898,7 @@ object DecisionTree extends Serializable with Logging { } }.maxBy(_._2.gain) - require(predict.isDefined, "must calculate predict for each node") + assert(predict.isDefined, "must calculate predict for each node") (bestSplit, bestSplitStats, predict.get) } 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 31d1e8ac30eea..caaccbfb8ad16 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 @@ -77,6 +77,8 @@ class Strategy ( } require(minInstancesPerNode >= 1, s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") + require(maxMemoryInMB <= 10240, + s"DecisionTree Strategy requires maxMemoryInMB <= 10240, but was given $maxMemoryInMB") val isMulticlassClassification = algo == Classification && numClassesForClassification > 2 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 6fac2be2797bc..d8476b5cd7bc7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -17,18 +17,14 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.DeveloperApi - /** - * :: DeveloperApi :: * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) */ -@DeveloperApi private[tree] class Predict( val predict: Double, - val prob: Double = 0.0) extends Serializable{ + val prob: Double = 0.0) extends Serializable { override def toString = { "predict = %f, prob = %f".format(predict, prob) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 1bd7ea05c46c8..2b2e579b992f6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -714,8 +714,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(gain == InformationGainStats.invalidInformationGainStats) } - test("don't choose split that doesn't satisfy min instance per node requirements") { - // if a split doesn't satisfy min instances per node requirements, + test("do not choose split that does not satisfy min instance per node requirements") { + // if a split does not satisfy min instances per node requirements, // this split is invalid, even though the information gain of split is large. val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index ccc000ac70ba6..5b13ab682bbfc 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -138,7 +138,8 @@ class DecisionTree(object): @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=5, maxBins=32): + impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): """ Train a DecisionTreeModel for classification. @@ -154,6 +155,9 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, E.g., depth 0 means 1 leaf node. Depth 1 means 1 internal node + 2 leaf nodes. :param maxBins: Number of bins used for finding splits at each node. + :param minInstancesPerNode: Min number of instances required at child nodes to create + the parent split + :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel """ sc = data.context @@ -164,13 +168,14 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( dataBytes._jrdd, "classification", numClasses, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins) + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) dataBytes.unpersist() return DecisionTreeModel(sc, model) @staticmethod def trainRegressor(data, categoricalFeaturesInfo, - impurity="variance", maxDepth=5, maxBins=32): + impurity="variance", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): """ Train a DecisionTreeModel for regression. @@ -185,6 +190,9 @@ def trainRegressor(data, categoricalFeaturesInfo, E.g., depth 0 means 1 leaf node. Depth 1 means 1 internal node + 2 leaf nodes. :param maxBins: Number of bins used for finding splits at each node. + :param minInstancesPerNode: Min number of instances required at child nodes to create + the parent split + :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel """ sc = data.context @@ -195,7 +203,7 @@ def trainRegressor(data, categoricalFeaturesInfo, model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( dataBytes._jrdd, "regression", 0, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins) + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) dataBytes.unpersist() return DecisionTreeModel(sc, model) From da33acb8b681eca5e787d546fe922af76a151398 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 15 Sep 2014 18:57:25 -0700 Subject: [PATCH 76/76] [SPARK-2951] [PySpark] support unpickle array.array for Python 2.6 Pyrolite can not unpickle array.array which pickled by Python 2.6, this patch fix it by extend Pyrolite. There is a bug in Pyrolite when unpickle array of float/double, this patch workaround it by reverse the endianness for float/double. This workaround should be removed after Pyrolite have a new release to fix this issue. I had send an PR to Pyrolite to fix it: https://github.com/irmen/Pyrolite/pull/11 Author: Davies Liu Closes #2365 from davies/pickle and squashes the following commits: f44f771 [Davies Liu] enable tests about array 3908f5c [Davies Liu] Merge branch 'master' into pickle c77c87b [Davies Liu] cleanup debugging code 60e4e2f [Davies Liu] support unpickle array.array for Python 2.6 --- .../apache/spark/api/python/SerDeUtil.scala | 51 +++++++++++++++++++ python/pyspark/context.py | 1 + python/pyspark/tests.py | 2 - 3 files changed, 52 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index efc9009c088a8..6668797f5f8be 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -17,6 +17,8 @@ package org.apache.spark.api.python +import java.nio.ByteOrder + import scala.collection.JavaConversions._ import scala.util.Failure import scala.util.Try @@ -28,6 +30,55 @@ import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ private[python] object SerDeUtil extends Logging { + // Unpickle array.array generated by Python 2.6 + class ArrayConstructor extends net.razorvine.pickle.objects.ArrayConstructor { + // /* Description of types */ + // static struct arraydescr descriptors[] = { + // {'c', sizeof(char), c_getitem, c_setitem}, + // {'b', sizeof(char), b_getitem, b_setitem}, + // {'B', sizeof(char), BB_getitem, BB_setitem}, + // #ifdef Py_USING_UNICODE + // {'u', sizeof(Py_UNICODE), u_getitem, u_setitem}, + // #endif + // {'h', sizeof(short), h_getitem, h_setitem}, + // {'H', sizeof(short), HH_getitem, HH_setitem}, + // {'i', sizeof(int), i_getitem, i_setitem}, + // {'I', sizeof(int), II_getitem, II_setitem}, + // {'l', sizeof(long), l_getitem, l_setitem}, + // {'L', sizeof(long), LL_getitem, LL_setitem}, + // {'f', sizeof(float), f_getitem, f_setitem}, + // {'d', sizeof(double), d_getitem, d_setitem}, + // {'\0', 0, 0, 0} /* Sentinel */ + // }; + // TODO: support Py_UNICODE with 2 bytes + // FIXME: unpickle array of float is wrong in Pyrolite, so we reverse the + // machine code for float/double here to workaround it. + // we should fix this after Pyrolite fix them + val machineCodes: Map[Char, Int] = if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { + Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 3, 'h' -> 5, 'I' -> 7, 'i' -> 9, + 'L' -> 11, 'l' -> 13, 'f' -> 14, 'd' -> 16, 'u' -> 21 + ) + } else { + Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 2, 'h' -> 4, 'I' -> 6, 'i' -> 8, + 'L' -> 10, 'l' -> 12, 'f' -> 15, 'd' -> 17, 'u' -> 20 + ) + } + override def construct(args: Array[Object]): Object = { + if (args.length == 1) { + construct(args ++ Array("")) + } else if (args.length == 2 && args(1).isInstanceOf[String]) { + val typecode = args(0).asInstanceOf[String].charAt(0) + val data: String = args(1).asInstanceOf[String] + construct(typecode, machineCodes(typecode), data.getBytes("ISO-8859-1")) + } else { + super.construct(args) + } + } + } + + def initialize() = { + Unpickler.registerConstructor("array", "array", new ArrayConstructor()) + } private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { val pickle = new Pickler diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 3ab98e262df31..ea28e8cd8c89f 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -214,6 +214,7 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile + SparkContext._jvm.SerDeUtil.initialize() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f3309a20fcffb..f255b44359fec 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -956,8 +956,6 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) - @unittest.skipIf(sys.version_info[:2] <= (2, 6) or python_implementation() == "PyPy", - "Skipped on 2.6 and PyPy until SPARK-2951 is fixed") def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays