Skip to content

Commit 0dafab5

Browse files
committed
Revert "[SPARK-30659][ML][PYSPARK] LogisticRegression blockify input vectors"
This reverts commit 073ce12.
1 parent 255ca22 commit 0dafab5

File tree

12 files changed

+113
-350
lines changed

12 files changed

+113
-350
lines changed

mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -157,7 +157,7 @@ class LinearSVC @Since("2.2.0") (
157157

158158
/**
159159
* Set block size for stacking input data in matrices.
160-
* Default is 1024.
160+
* Default is 4096.
161161
*
162162
* @group expertSetParam
163163
*/
@@ -240,7 +240,7 @@ class LinearSVC @Since("2.2.0") (
240240
.persist(StorageLevel.MEMORY_AND_DISK)
241241
.setName(s"training dataset (blockSize=${$(blockSize)})")
242242

243-
val getAggregatorFunc = new HingeAggregator(numFeatures, $(fitIntercept))(_)
243+
val getAggregatorFunc = new HingeAggregator(numFeatures, $(fitIntercept), $(blockSize))(_)
244244
val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization,
245245
$(aggregationDepth))
246246

mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala

Lines changed: 21 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path
2828
import org.apache.spark.SparkException
2929
import org.apache.spark.annotation.Since
3030
import org.apache.spark.internal.Logging
31-
import org.apache.spark.ml.feature.{Instance, InstanceBlock}
31+
import org.apache.spark.ml.feature.Instance
3232
import org.apache.spark.ml.linalg._
3333
import org.apache.spark.ml.optim.aggregator.LogisticAggregator
3434
import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction}
@@ -50,8 +50,7 @@ import org.apache.spark.util.VersionUtils
5050
*/
5151
private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams
5252
with HasRegParam with HasElasticNetParam with HasMaxIter with HasFitIntercept with HasTol
53-
with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth
54-
with HasBlockSize {
53+
with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth {
5554

5655
import org.apache.spark.ml.classification.LogisticRegression.supportedFamilyNames
5756

@@ -431,15 +430,6 @@ class LogisticRegression @Since("1.2.0") (
431430
@Since("2.2.0")
432431
def setUpperBoundsOnIntercepts(value: Vector): this.type = set(upperBoundsOnIntercepts, value)
433432

434-
/**
435-
* Set block size for stacking input data in matrices.
436-
* Default is 1024.
437-
*
438-
* @group expertSetParam
439-
*/
440-
@Since("3.0.0")
441-
def setBlockSize(value: Int): this.type = set(blockSize, value)
442-
443433
private def assertBoundConstrainedOptimizationParamsValid(
444434
numCoefficientSets: Int,
445435
numFeatures: Int): Unit = {
@@ -492,17 +482,24 @@ class LogisticRegression @Since("1.2.0") (
492482
this
493483
}
494484

495-
override protected[spark] def train(
496-
dataset: Dataset[_]): LogisticRegressionModel = instrumented { instr =>
485+
override protected[spark] def train(dataset: Dataset[_]): LogisticRegressionModel = {
486+
val handlePersistence = dataset.storageLevel == StorageLevel.NONE
487+
train(dataset, handlePersistence)
488+
}
489+
490+
protected[spark] def train(
491+
dataset: Dataset[_],
492+
handlePersistence: Boolean): LogisticRegressionModel = instrumented { instr =>
493+
val instances = extractInstances(dataset)
494+
495+
if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK)
496+
497497
instr.logPipelineStage(this)
498498
instr.logDataset(dataset)
499499
instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol,
500500
probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol,
501501
fitIntercept)
502502

503-
val sc = dataset.sparkSession.sparkContext
504-
val instances = extractInstances(dataset)
505-
506503
val (summarizer, labelSummarizer) = instances.treeAggregate(
507504
(Summarizer.createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))(
508505
seqOp = (c: (SummarizerBuffer, MultiClassSummarizer), instance: Instance) =>
@@ -585,9 +582,8 @@ class LogisticRegression @Since("1.2.0") (
585582
s"dangerous ground, so the algorithm may not converge.")
586583
}
587584

588-
val featuresMean = summarizer.mean.compressed
589-
val featuresStd = summarizer.std.compressed
590-
val bcFeaturesStd = sc.broadcast(featuresStd)
585+
val featuresMean = summarizer.mean.toArray
586+
val featuresStd = summarizer.std.toArray
591587

592588
if (!$(fitIntercept) && (0 until numFeatures).exists { i =>
593589
featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) {
@@ -599,7 +595,8 @@ class LogisticRegression @Since("1.2.0") (
599595
val regParamL1 = $(elasticNetParam) * $(regParam)
600596
val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam)
601597

602-
val getAggregatorFunc = new LogisticAggregator(numFeatures, numClasses, $(fitIntercept),
598+
val bcFeaturesStd = instances.context.broadcast(featuresStd)
599+
val getAggregatorFunc = new LogisticAggregator(bcFeaturesStd, numClasses, $(fitIntercept),
603600
multinomial = isMultinomial)(_)
604601
val getFeaturesStd = (j: Int) => if (j >= 0 && j < numCoefficientSets * numFeatures) {
605602
featuresStd(j / numCoefficientSets)
@@ -615,21 +612,7 @@ class LogisticRegression @Since("1.2.0") (
615612
None
616613
}
617614

618-
val standardized = instances.map {
619-
case Instance(label, weight, features) =>
620-
val featuresStd = bcFeaturesStd.value
621-
val array = Array.ofDim[Double](numFeatures)
622-
features.foreachNonZero { (i, v) =>
623-
val std = featuresStd(i)
624-
if (std != 0) array(i) = v / std
625-
}
626-
Instance(label, weight, Vectors.dense(array))
627-
}
628-
val blocks = InstanceBlock.blokify(standardized, $(blockSize))
629-
.persist(StorageLevel.MEMORY_AND_DISK)
630-
.setName(s"training dataset (blockSize=${$(blockSize)})")
631-
632-
val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization,
615+
val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization,
633616
$(aggregationDepth))
634617

635618
val numCoeffsPlusIntercepts = numFeaturesPlusIntercept * numCoefficientSets
@@ -823,7 +806,6 @@ class LogisticRegression @Since("1.2.0") (
823806
state = states.next()
824807
arrayBuilder += state.adjustedValue
825808
}
826-
blocks.unpersist()
827809
bcFeaturesStd.destroy()
828810

829811
if (state == null) {
@@ -893,6 +875,8 @@ class LogisticRegression @Since("1.2.0") (
893875
}
894876
}
895877

878+
if (handlePersistence) instances.unpersist()
879+
896880
val model = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector,
897881
numClasses, isMultinomial))
898882

mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala

Lines changed: 24 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,8 @@ import org.apache.spark.ml.linalg._
3535
*/
3636
private[ml] class HingeAggregator(
3737
numFeatures: Int,
38-
fitIntercept: Boolean)(bcCoefficients: Broadcast[Vector])
38+
fitIntercept: Boolean,
39+
blockSize: Int = 4096)(bcCoefficients: Broadcast[Vector])
3940
extends DifferentiableLossAggregator[InstanceBlock, HingeAggregator] {
4041

4142
private val numFeaturesPlusIntercept: Int = if (fitIntercept) numFeatures + 1 else numFeatures
@@ -54,6 +55,20 @@ private[ml] class HingeAggregator(
5455
}
5556
}
5657

58+
@transient private lazy val intercept =
59+
if (fitIntercept) coefficientsArray(numFeatures) else 0.0
60+
61+
@transient private lazy val linearGradSumVec = {
62+
if (fitIntercept) {
63+
new DenseVector(Array.ofDim[Double](numFeatures))
64+
} else {
65+
null
66+
}
67+
}
68+
69+
@transient private lazy val auxiliaryVec =
70+
new DenseVector(Array.ofDim[Double](blockSize))
71+
5772

5873
/**
5974
* Add a new training instance to this HingeAggregator, and update the loss and gradient
@@ -123,14 +138,19 @@ private[ml] class HingeAggregator(
123138
val localGradientSumArray = gradientSumArray
124139

125140
// vec here represents dotProducts
126-
val vec = if (fitIntercept && coefficientsArray.last != 0) {
127-
val intercept = coefficientsArray.last
128-
new DenseVector(Array.fill(size)(intercept))
141+
val vec = if (size == blockSize) {
142+
auxiliaryVec
129143
} else {
144+
// the last block within one partition may be of size less than blockSize
130145
new DenseVector(Array.ofDim[Double](size))
131146
}
132147

133148
if (fitIntercept) {
149+
var i = 0
150+
while (i < size) {
151+
vec.values(i) = intercept
152+
i += 1
153+
}
134154
BLAS.gemv(1.0, block.matrix, linear, 1.0, vec)
135155
} else {
136156
BLAS.gemv(1.0, block.matrix, linear, 0.0, vec)
@@ -165,9 +185,6 @@ private[ml] class HingeAggregator(
165185
if (vec.values.forall(_ == 0)) return this
166186

167187
if (fitIntercept) {
168-
// localGradientSumArray is of size numFeatures+1, so can not
169-
// be directly used as the output of BLAS.gemv
170-
val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures))
171188
BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec)
172189
linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v }
173190
localGradientSumArray(numFeatures) += vec.values.sum

0 commit comments

Comments
 (0)