-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-30642][ML][PYSPARK] LinearSVC blockify input vectors #28349
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
8ad507b
init
zhengruifeng 21fe97c
update version
zhengruifeng f969e4a
update doc
zhengruifeng 68dbc35
fix py
zhengruifeng 5e6e5e7
nit
zhengruifeng a74da0c
nit
zhengruifeng f8fbab7
nit
zhengruifeng 58c0a1e
nit
zhengruifeng 4ed1227
address comments
zhengruifeng e02a86e
fix doc
zhengruifeng e8abb4b
remove some transient lazy variables
zhengruifeng File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,21 +26,23 @@ import org.apache.hadoop.fs.Path | |
| import org.apache.spark.SparkException | ||
| import org.apache.spark.annotation.Since | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.ml.feature._ | ||
| import org.apache.spark.ml.linalg._ | ||
| import org.apache.spark.ml.optim.aggregator.HingeAggregator | ||
| import org.apache.spark.ml.optim.aggregator._ | ||
| import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} | ||
| import org.apache.spark.ml.param._ | ||
| import org.apache.spark.ml.param.shared._ | ||
| import org.apache.spark.ml.stat._ | ||
| import org.apache.spark.ml.util._ | ||
| import org.apache.spark.ml.util.Instrumentation.instrumented | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.{Dataset, Row} | ||
| import org.apache.spark.storage.StorageLevel | ||
|
|
||
| /** Params for linear SVM Classifier. */ | ||
| private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam | ||
| with HasMaxIter with HasFitIntercept with HasTol with HasStandardization with HasWeightCol | ||
| with HasAggregationDepth with HasThreshold { | ||
| with HasAggregationDepth with HasThreshold with HasBlockSize { | ||
|
|
||
| /** | ||
| * Param for threshold in binary classification prediction. | ||
|
|
@@ -154,31 +156,65 @@ class LinearSVC @Since("2.2.0") ( | |
| def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) | ||
| setDefault(aggregationDepth -> 2) | ||
|
|
||
| /** | ||
| * Set block size for stacking input data in matrices. | ||
| * If blockSize == 1, then stacking will be skipped, and each vector is treated individually; | ||
| * If blockSize > 1, then vectors will be stacked to blocks, and high-level BLAS routines | ||
| * will be used if possible (for example, GEMV instead of DOT, GEMM instead of GEMV). | ||
| * Recommended size is between 10 and 1000. An appropriate choice of the block size depends | ||
| * on the sparsity and dim of input datasets, the underlying BLAS implementation (for example, | ||
| * f2jBLAS, OpenBLAS, intel MKL) and its configuration (for example, number of threads). | ||
| * Note that existing BLAS implementations are mainly optimized for dense matrices, if the | ||
| * input dataset is sparse, stacking may bring no performance gain, the worse is possible | ||
| * performance regression. | ||
| * Default is 1. | ||
| * | ||
| * @group expertSetParam | ||
| */ | ||
| @Since("3.1.0") | ||
| def setBlockSize(value: Int): this.type = set(blockSize, value) | ||
| setDefault(blockSize -> 1) | ||
|
|
||
| @Since("2.2.0") | ||
| override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) | ||
|
|
||
| override protected def train(dataset: Dataset[_]): LinearSVCModel = instrumented { instr => | ||
| val handlePersistence = dataset.storageLevel == StorageLevel.NONE | ||
|
|
||
| val instances = extractInstances(dataset) | ||
| if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) | ||
|
|
||
| instr.logPipelineStage(this) | ||
| instr.logDataset(dataset) | ||
| instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, | ||
| regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth) | ||
| regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth, blockSize) | ||
|
|
||
| val instances = extractInstances(dataset) | ||
| .setName("training instances") | ||
|
|
||
| val (summarizer, labelSummarizer) = | ||
| val (summarizer, labelSummarizer) = if ($(blockSize) == 1) { | ||
| if (dataset.storageLevel == StorageLevel.NONE) { | ||
| instances.persist(StorageLevel.MEMORY_AND_DISK) | ||
| } | ||
| Summarizer.getClassificationSummarizers(instances, $(aggregationDepth)) | ||
| instr.logNumExamples(summarizer.count) | ||
| instr.logNamedValue("lowestLabelWeight", labelSummarizer.histogram.min.toString) | ||
| instr.logNamedValue("highestLabelWeight", labelSummarizer.histogram.max.toString) | ||
| instr.logSumOfWeights(summarizer.weightSum) | ||
| } else { | ||
| // instances will be standardized and converted to blocks, so no need to cache instances. | ||
| Summarizer.getClassificationSummarizers(instances, $(aggregationDepth), | ||
| Seq("mean", "std", "count", "numNonZeros")) | ||
| } | ||
|
|
||
| val histogram = labelSummarizer.histogram | ||
| val numInvalid = labelSummarizer.countInvalid | ||
| val numFeatures = summarizer.mean.size | ||
| val numFeaturesPlusIntercept = if (getFitIntercept) numFeatures + 1 else numFeatures | ||
|
|
||
| instr.logNumExamples(summarizer.count) | ||
| instr.logNamedValue("lowestLabelWeight", labelSummarizer.histogram.min.toString) | ||
| instr.logNamedValue("highestLabelWeight", labelSummarizer.histogram.max.toString) | ||
| instr.logSumOfWeights(summarizer.weightSum) | ||
| if ($(blockSize) > 1) { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it is up to the end user to choose whether high-level blas is used and which BLAS lib is used. |
||
| val scale = 1.0 / summarizer.count / numFeatures | ||
| val sparsity = 1 - summarizer.numNonzeros.toArray.map(_ * scale).sum | ||
| instr.logNamedValue("sparsity", sparsity.toString) | ||
| if (sparsity > 0.5) { | ||
| instr.logWarning(s"sparsity of input dataset is $sparsity, " + | ||
| s"which may hurt performance in high-level BLAS.") | ||
| } | ||
| } | ||
|
|
||
| val numClasses = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { | ||
| case Some(n: Int) => | ||
|
|
@@ -192,77 +228,113 @@ class LinearSVC @Since("2.2.0") ( | |
| instr.logNumClasses(numClasses) | ||
| instr.logNumFeatures(numFeatures) | ||
|
|
||
| val (coefficientVector, interceptVector, objectiveHistory) = { | ||
| if (numInvalid != 0) { | ||
| val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " + | ||
| s"Found $numInvalid invalid labels." | ||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| } | ||
| if (numInvalid != 0) { | ||
| val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " + | ||
| s"Found $numInvalid invalid labels." | ||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| } | ||
|
|
||
| val featuresStd = summarizer.std.toArray | ||
| val getFeaturesStd = (j: Int) => featuresStd(j) | ||
| val regParamL2 = $(regParam) | ||
| val bcFeaturesStd = instances.context.broadcast(featuresStd) | ||
| val regularization = if (regParamL2 != 0.0) { | ||
| val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures | ||
| Some(new L2Regularization(regParamL2, shouldApply, | ||
| if ($(standardization)) None else Some(getFeaturesStd))) | ||
| } else { | ||
| None | ||
| } | ||
| val featuresStd = summarizer.std.toArray | ||
| val getFeaturesStd = (j: Int) => featuresStd(j) | ||
| val regularization = if ($(regParam) != 0.0) { | ||
| val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures | ||
| Some(new L2Regularization($(regParam), shouldApply, | ||
| if ($(standardization)) None else Some(getFeaturesStd))) | ||
| } else None | ||
|
|
||
| def regParamL1Fun = (index: Int) => 0.0 | ||
| val optimizer = new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol)) | ||
|
|
||
| /* | ||
| The coefficients are trained in the scaled space; we're converting them back to | ||
| the original space. | ||
| Note that the intercept in scaled space and original space is the same; | ||
| as a result, no scaling is needed. | ||
| */ | ||
| val (rawCoefficients, objectiveHistory) = if ($(blockSize) == 1) { | ||
| trainOnRows(instances, featuresStd, regularization, optimizer) | ||
| } else { | ||
| trainOnBlocks(instances, featuresStd, regularization, optimizer) | ||
| } | ||
| if (instances.getStorageLevel != StorageLevel.NONE) instances.unpersist() | ||
|
|
||
| val getAggregatorFunc = new HingeAggregator(bcFeaturesStd, $(fitIntercept))(_) | ||
| val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, | ||
| $(aggregationDepth)) | ||
| if (rawCoefficients == null) { | ||
| val msg = s"${optimizer.getClass.getName} failed." | ||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| } | ||
|
|
||
| def regParamL1Fun = (index: Int) => 0D | ||
| val optimizer = new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol)) | ||
| val initialCoefWithIntercept = Vectors.zeros(numFeaturesPlusIntercept) | ||
| val coefficientArray = Array.tabulate(numFeatures) { i => | ||
| if (featuresStd(i) != 0.0) rawCoefficients(i) / featuresStd(i) else 0.0 | ||
| } | ||
| val intercept = if ($(fitIntercept)) rawCoefficients.last else 0.0 | ||
| copyValues(new LinearSVCModel(uid, Vectors.dense(coefficientArray), intercept)) | ||
| } | ||
|
|
||
| val states = optimizer.iterations(new CachedDiffFunction(costFun), | ||
| initialCoefWithIntercept.asBreeze.toDenseVector) | ||
| private def trainOnRows( | ||
| instances: RDD[Instance], | ||
| featuresStd: Array[Double], | ||
| regularization: Option[L2Regularization], | ||
| optimizer: BreezeOWLQN[Int, BDV[Double]]): (Array[Double], Array[Double]) = { | ||
| val numFeatures = featuresStd.length | ||
| val numFeaturesPlusIntercept = if ($(fitIntercept)) numFeatures + 1 else numFeatures | ||
|
|
||
| val bcFeaturesStd = instances.context.broadcast(featuresStd) | ||
| val getAggregatorFunc = new HingeAggregator(bcFeaturesStd, $(fitIntercept))(_) | ||
| val costFun = new RDDLossFunction(instances, getAggregatorFunc, | ||
| regularization, $(aggregationDepth)) | ||
|
|
||
| val states = optimizer.iterations(new CachedDiffFunction(costFun), | ||
| Vectors.zeros(numFeaturesPlusIntercept).asBreeze.toDenseVector) | ||
|
|
||
| val arrayBuilder = mutable.ArrayBuilder.make[Double] | ||
| var state: optimizer.State = null | ||
| while (states.hasNext) { | ||
| state = states.next() | ||
| arrayBuilder += state.adjustedValue | ||
| } | ||
| bcFeaturesStd.destroy() | ||
|
|
||
| val scaledObjectiveHistory = mutable.ArrayBuilder.make[Double] | ||
| var state: optimizer.State = null | ||
| while (states.hasNext) { | ||
| state = states.next() | ||
| scaledObjectiveHistory += state.adjustedValue | ||
| } | ||
| (if (state != null) state.x.toArray else null, arrayBuilder.result) | ||
| } | ||
|
|
||
| bcFeaturesStd.destroy() | ||
| if (state == null) { | ||
| val msg = s"${optimizer.getClass.getName} failed." | ||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| } | ||
| private def trainOnBlocks( | ||
| instances: RDD[Instance], | ||
| featuresStd: Array[Double], | ||
| regularization: Option[L2Regularization], | ||
| optimizer: BreezeOWLQN[Int, BDV[Double]]): (Array[Double], Array[Double]) = { | ||
| val numFeatures = featuresStd.length | ||
| val numFeaturesPlusIntercept = if ($(fitIntercept)) numFeatures + 1 else numFeatures | ||
|
|
||
| /* | ||
| The coefficients are trained in the scaled space; we're converting them back to | ||
| the original space. | ||
| Note that the intercept in scaled space and original space is the same; | ||
| as a result, no scaling is needed. | ||
| */ | ||
| val rawCoefficients = state.x.toArray | ||
| val coefficientArray = Array.tabulate(numFeatures) { i => | ||
| if (featuresStd(i) != 0.0) { | ||
| rawCoefficients(i) / featuresStd(i) | ||
| } else { | ||
| 0.0 | ||
| } | ||
| } | ||
| val bcFeaturesStd = instances.context.broadcast(featuresStd) | ||
|
|
||
| val intercept = if ($(fitIntercept)) { | ||
| rawCoefficients(numFeaturesPlusIntercept - 1) | ||
| } else { | ||
| 0.0 | ||
| } | ||
| (Vectors.dense(coefficientArray), intercept, scaledObjectiveHistory.result()) | ||
| val standardized = instances.mapPartitions { iter => | ||
| val inverseStd = bcFeaturesStd.value.map { std => if (std != 0) 1.0 / std else 0.0 } | ||
| val func = StandardScalerModel.getTransformFunc(Array.empty, inverseStd, false, true) | ||
| iter.map { case Instance(label, weight, vec) => Instance(label, weight, func(vec)) } | ||
| } | ||
| val blocks = InstanceBlock.blokify(standardized, $(blockSize)) | ||
| .persist(StorageLevel.MEMORY_AND_DISK) | ||
| .setName(s"training dataset (blockSize=${$(blockSize)})") | ||
|
|
||
| val getAggregatorFunc = new BlockHingeAggregator($(fitIntercept))(_) | ||
| val costFun = new RDDLossFunction(blocks, getAggregatorFunc, | ||
| regularization, $(aggregationDepth)) | ||
|
|
||
| val states = optimizer.iterations(new CachedDiffFunction(costFun), | ||
| Vectors.zeros(numFeaturesPlusIntercept).asBreeze.toDenseVector) | ||
|
|
||
| val arrayBuilder = mutable.ArrayBuilder.make[Double] | ||
| var state: optimizer.State = null | ||
| while (states.hasNext) { | ||
| state = states.next() | ||
| arrayBuilder += state.adjustedValue | ||
| } | ||
| blocks.unpersist() | ||
| bcFeaturesStd.destroy() | ||
|
|
||
| if (handlePersistence) instances.unpersist() | ||
|
|
||
| copyValues(new LinearSVCModel(uid, coefficientVector, interceptVector)) | ||
| (if (state != null) state.x.toArray else null, arrayBuilder.result) | ||
| } | ||
| } | ||
|
|
||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We might provide a little more comment about what this does. Increasing it increases performance, but at the risk of what, slowing down on sparse input?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The choice of size needs tuning, it depends on dataset sparsity and numFeatures, Increasing it may not always increases performance.