From c67e79ecd77d93b67eeefb76e1054ef4092f9553 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 11:00:21 +0800 Subject: [PATCH 01/29] Add NaiveBayes skeleton code --- .../ml/classification/NaiveBayes.scala | 651 ++++++++++++++++++ .../ml/classification/NaiveBayesImpl.scala | 33 + 2 files changed, 684 insertions(+) create mode 100644 mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala new file mode 100644 index 000000000..b5828939f --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -0,0 +1,651 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.apache.spark.annotation.Since +import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.HasWeightCol +import org.apache.spark.ml.stat.Summarizer +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.VersionUtils + +/** + * Params for Naive Bayes Classifiers. + */ +private[classification] trait NaiveBayesParams extends PredictorParams with HasWeightCol { + + /** + * The smoothing parameter. + * (default = 1.0). + * @group param + */ + final val smoothing: DoubleParam = new DoubleParam(this, "smoothing", "The smoothing parameter.", + ParamValidators.gtEq(0)) + + /** @group getParam */ + final def getSmoothing: Double = $(smoothing) + + /** + * The model type which is a string (case-sensitive). + * Supported options: "multinomial", "complement", "bernoulli", "gaussian". + * (default = multinomial) + * @group param + */ + final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + + "which is a string (case-sensitive). Supported options: multinomial (default), complement, " + + "bernoulli and gaussian.", + ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) + + /** @group getParam */ + final def getModelType: String = $(modelType) +} + +// scalastyle:off line.size.limit +/** + * Naive Bayes Classifiers. + * It supports Multinomial NB + * (see + * here) + * which can handle finitely supported discrete data. For example, by converting documents into + * TF-IDF vectors, it can be used for document classification. By making every vector a + * binary (0/1) data, it can also be used as Bernoulli NB + * (see + * here). + * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + * Since 3.0.0, it supports Complement NB which is an adaptation of the Multinomial NB. Specifically, + * Complement NB uses statistics from the complement of each class to compute the model's coefficients + * The inventors of Complement NB show empirically that the parameter estimates for CNB are more stable + * than those for Multinomial NB. Like Multinomial NB, the input feature values for Complement NB must + * be nonnegative. + * Since 3.0.0, it also supports Gaussian NB + * (see + * here) + * which can handle continuous data. + */ +// scalastyle:on line.size.limit +@Since("1.5.0") +class NaiveBayes @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends ProbabilisticClassifier[Vector, NaiveBayes, NaiveBayesModel] + with NaiveBayesParams with DefaultParamsWritable { + + import NaiveBayes._ + + @Since("1.5.0") + def this() = this(Identifiable.randomUID("nb")) + + /** + * Set the smoothing parameter. + * Default is 1.0. + * @group setParam + */ + @Since("1.5.0") + def setSmoothing(value: Double): this.type = set(smoothing, value) + setDefault(smoothing -> 1.0) + + /** + * Set the model type using a string (case-sensitive). + * Supported options: "multinomial", "complement", "bernoulli", and "gaussian". + * Default is "multinomial" + * @group setParam + */ + @Since("1.5.0") + def setModelType(value: String): this.type = set(modelType, value) + setDefault(modelType -> Multinomial) + + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("2.1.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + override protected def train(dataset: Dataset[_]): NaiveBayesModel = { + trainWithLabelCheck(dataset, positiveLabel = true) + } + + /** + * ml assumes input labels in range [0, numClasses). But this implementation + * is also called by mllib NaiveBayes which allows other kinds of input labels + * such as {-1, +1}. `positiveLabel` is used to determine whether the label + * should be checked and it should be removed when we remove mllib NaiveBayes. + */ + private[spark] def trainWithLabelCheck( + dataset: Dataset[_], + positiveLabel: Boolean): NaiveBayesModel = instrumented { instr => + instr.logPipelineStage(this) + instr.logDataset(dataset) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) + + if (positiveLabel && isDefined(thresholds)) { + val numClasses = getNumClasses(dataset) + instr.logNumClasses(numClasses) + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + + $(modelType) match { + case Multinomial => + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + dataset.sparkSession.sparkContext) + val handleWeight = (isDefined(weightCol) && $(weightCol).nonEmpty) + if (isPlatformSupported && !handleWeight) { + trainNaiveBayesDAL(dataset, instr) + } else { + trainDiscreteImpl(dataset, instr) + } + case Bernoulli | Complement => + trainDiscreteImpl(dataset, instr) + case Gaussian => + trainGaussianImpl(dataset, instr) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + } + } + + private def trainNaiveBayesDAL(dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + + val sc = dataset.sparkSession.sparkContext + val executor_num = Utils.sparkExecutorNum(sc) + val executor_cores = Utils.sparkExecutorCores() + + logInfo(s"NaiveBayesDAL fit using $executor_num Executors") + + val features: RDD[Vector] = dataset + .select(DatasetUtils.columnToVector(dataset, getFeaturesCol)).rdd.map { + case Row(feature: Vector) => feature + } + val labels: RDD[Double] = dataset.select(col(getLabelCol)).rdd.map { + case Row(label: Double) => label + } + + val model = new NaiveBayesImpl( + executor_num, executor_cores).train(features, labels, Some(instr)) + model + } + + private def trainDiscreteImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ + + val validateUDF = $(modelType) match { + case Multinomial | Complement => + udf { vector: Vector => requireNonnegativeValues(vector); vector } + case Bernoulli => + udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } + } + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + // Aggregates term frequencies per label. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("sum", "count") + .summary(validateUDF(col($(featuresCol))), w).as("summary")) + .select($(labelCol), "weightSum", "summary.sum", "summary.count") + .as[(Double, Double, Vector, Long)] + .collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + val numSamples = aggregated.map(_._4).sum + instr.logNumExamples(numSamples) + val numLabels = aggregated.length + instr.logNumClasses(numLabels) + val numDocuments = aggregated.map(_._2).sum + instr.logSumOfWeights(numDocuments) + + val labelArray = new Array[Double](numLabels) + val piArray = new Array[Double](numLabels) + val thetaArray = new Array[Double](numLabels * numFeatures) + + val aggIter = $(modelType) match { + case Multinomial | Bernoulli => aggregated.iterator + case Complement => + val featureSum = Vectors.zeros(numFeatures) + aggregated.foreach { case (_, _, sumTermFreqs, _) => + BLAS.axpy(1.0, sumTermFreqs, featureSum) + } + aggregated.iterator.map { case (label, n, sumTermFreqs, count) => + val comp = featureSum.copy + BLAS.axpy(-1.0, sumTermFreqs, comp) + (label, n, comp, count) + } + } + + val lambda = $(smoothing) + val piLogDenom = math.log(numDocuments + numLabels * lambda) + var i = 0 + aggIter.foreach { case (label, n, sumTermFreqs, _) => + labelArray(i) = label + piArray(i) = math.log(n + lambda) - piLogDenom + val thetaLogDenom = $(modelType) match { + case Multinomial | Complement => + math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) + case Bernoulli => math.log(n + 2.0 * lambda) + } + var j = 0 + val offset = i * numFeatures + while (j < numFeatures) { + thetaArray(offset + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + j += 1 + } + i += 1 + } + + val pi = Vectors.dense(piArray) + $(modelType) match { + case Multinomial | Bernoulli => + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, Matrices.zeros(0, 0)) + .setOldLabels(labelArray) + case Complement => + // Since the CNB compute the coefficient in a complement way. + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray.map(v => -v), true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, Matrices.zeros(0, 0)) + } + } + + private def trainGaussianImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + // Aggregates mean vector and square-sum vector per label. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "normL2") + .summary(col($(featuresCol)), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.normL2") + .as[(Double, Double, Vector, Vector)] + .map { case (label, weightSum, mean, normL2) => + (label, weightSum, mean, Vectors.dense(normL2.toArray.map(v => v * v))) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + + val numLabels = aggregated.length + instr.logNumClasses(numLabels) + + val numInstances = aggregated.map(_._2).sum + instr.logSumOfWeights(numInstances) + + // If the ratio of data variance between dimensions is too small, it + // will cause numerical errors. To address this, we artificially + // boost the variance by epsilon, a small fraction of the standard + // deviation of the largest dimension. + // Refer to scikit-learn's implementation + // [https://github.com/scikit-learn/scikit-learn/blob/0.21.X/sklearn/naive_bayes.py#L348] + // and discussion [https://github.com/scikit-learn/scikit-learn/pull/5349] for detail. + val epsilon = Iterator.range(0, numFeatures).map { j => + var globalSum = 0.0 + var globalSqrSum = 0.0 + aggregated.foreach { case (_, weightSum, mean, squareSum) => + globalSum += mean(j) * weightSum + globalSqrSum += squareSum(j) + } + globalSqrSum / numInstances - + globalSum * globalSum / numInstances / numInstances + }.max * 1e-9 + + val piArray = new Array[Double](numLabels) + + // thetaArray in Gaussian NB store the means of features per label + val thetaArray = new Array[Double](numLabels * numFeatures) + + // thetaArray in Gaussian NB store the variances of features per label + val sigmaArray = new Array[Double](numLabels * numFeatures) + + var i = 0 + val logNumInstances = math.log(numInstances) + aggregated.foreach { case (_, weightSum, mean, squareSum) => + piArray(i) = math.log(weightSum) - logNumInstances + var j = 0 + val offset = i * numFeatures + while (j < numFeatures) { + val m = mean(j) + thetaArray(offset + j) = m + sigmaArray(offset + j) = epsilon + squareSum(j) / weightSum - m * m + j += 1 + } + i += 1 + } + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + val sigma = new DenseMatrix(numLabels, numFeatures, sigmaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, sigma.compressed) + } + + @Since("1.5.0") + override def copy(extra: ParamMap): NaiveBayes = defaultCopy(extra) +} + +@Since("1.6.0") +object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { + /** String name for multinomial model type. */ + private[classification] val Multinomial: String = "multinomial" + + /** String name for Bernoulli model type. */ + private[classification] val Bernoulli: String = "bernoulli" + + /** String name for Gaussian model type. */ + private[classification] val Gaussian: String = "gaussian" + + /** String name for Complement model type. */ + private[classification] val Complement: String = "complement" + + /* Set of modelTypes that NaiveBayes supports */ + private[classification] val supportedModelTypes = + Set(Multinomial, Bernoulli, Gaussian, Complement) + + private[ml] def requireNonnegativeValues(v: Vector): Unit = { + require(v.nonZeroIterator.forall(_._2 > 0.0), + s"Naive Bayes requires nonnegative feature values but found $v.") + } + + private[ml] def requireZeroOneBernoulliValues(v: Vector): Unit = { + require(v.nonZeroIterator.forall(_._2 == 1.0), + s"Bernoulli naive Bayes requires 0 or 1 feature values but found $v.") + } + + @Since("1.6.0") + override def load(path: String): NaiveBayes = super.load(path) +} + +/** + * Model produced by [[NaiveBayes]] + * + * @param pi log of class priors, whose dimension is C (number of classes) + * @param theta log of class conditional probabilities, whose dimension is C (number of classes) + * by D (number of features) + * @param sigma variance of each feature, whose dimension is C (number of classes) + * by D (number of features). This matrix is only available when modelType + * is set Gaussian. + */ +@Since("1.5.0") +class NaiveBayesModel private[ml] ( + @Since("1.5.0") override val uid: String, + @Since("2.0.0") val pi: Vector, + @Since("2.0.0") val theta: Matrix, + @Since("3.0.0") val sigma: Matrix) + extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] + with NaiveBayesParams with MLWritable { + + import NaiveBayes._ + + /** + * mllib NaiveBayes is a wrapper of ml implementation currently. + * Input labels of mllib could be {-1, +1} and mllib NaiveBayesModel exposes labels, + * both of which are different from ml, so we should store the labels sequentially + * to be called by mllib. This should be removed when we remove mllib NaiveBayes. + */ + private[spark] var oldLabels: Array[Double] = null + + private[spark] def setOldLabels(labels: Array[Double]): this.type = { + this.oldLabels = labels + this + } + + /** + * Bernoulli scoring requires log(condprob) if 1, log(1-condprob) if 0. + * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra + * application of this condition (in predict function). + */ + @transient private lazy val (thetaMinusNegTheta, piMinusThetaSum) = $(modelType) match { + case Bernoulli => + val thetaMinusNegTheta = theta.map(value => value - math.log1p(-math.exp(value))) + val negTheta = theta.map(value => math.log1p(-math.exp(value))) + val ones = new DenseVector(Array.fill(theta.numCols)(1.0)) + val piMinusThetaSum = pi.toDense.copy + BLAS.gemv(1.0, negTheta, ones, 1.0, piMinusThetaSum) + (thetaMinusNegTheta, piMinusThetaSum) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables thetaMinusNegTheta and negThetaSum should only be precomputed in Bernoulli NB.") + } + + /** + * Gaussian scoring requires sum of log(Variance). + * This precomputes sum of log(Variance) which are used for the linear algebra + * application of this condition (in predict function). + */ + @transient private lazy val logVarSum = $(modelType) match { + case Gaussian => + Array.tabulate(numClasses) { i => + Iterator.range(0, numFeatures).map { j => + math.log(sigma(i, j)) + }.sum + } + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables logVarSum should only be precomputed in Gaussian NB.") + } + + @Since("1.6.0") + override val numFeatures: Int = theta.numCols + + @Since("1.5.0") + override val numClasses: Int = pi.size + + private def multinomialCalculation(features: Vector) = { + requireNonnegativeValues(features) + val prob = pi.toDense.copy + BLAS.gemv(1.0, theta, features, 1.0, prob) + prob + } + + private def complementCalculation(features: Vector) = { + requireNonnegativeValues(features) + val probArray = theta.multiply(features).toArray + // the following lines equal to: + // val logSumExp = math.log(probArray.map(math.exp).sum) + // However, it easily returns Infinity/NaN values. + // Here follows 'scipy.special.logsumexp' (which is used in Scikit-Learn's ComplementNB) + // to compute the log of the sum of exponentials of elements in a numeric-stable way. + val max = probArray.max + var sumExp = 0.0 + var j = 0 + while (j < probArray.length) { + sumExp += math.exp(probArray(j) - max) + j += 1 + } + val logSumExp = math.log(sumExp) + max + + j = 0 + while (j < probArray.length) { + probArray(j) = probArray(j) - logSumExp + j += 1 + } + Vectors.dense(probArray) + } + + private def bernoulliCalculation(features: Vector) = { + requireZeroOneBernoulliValues(features) + val prob = piMinusThetaSum.copy + BLAS.gemv(1.0, thetaMinusNegTheta, features, 1.0, prob) + prob + } + + private def gaussianCalculation(features: Vector) = { + val prob = Array.ofDim[Double](numClasses) + var i = 0 + while (i < numClasses) { + var s = 0.0 + var j = 0 + while (j < numFeatures) { + val d = features(j) - theta(i, j) + s += d * d / sigma(i, j) + j += 1 + } + prob(i) = pi(i) - (s + logVarSum(i)) / 2 + i += 1 + } + Vectors.dense(prob) + } + + @transient private lazy val predictRawFunc = { + $(modelType) match { + case Multinomial => + features: Vector => multinomialCalculation(features) + case Complement => + features: Vector => complementCalculation(features) + case Bernoulli => + features: Vector => bernoulliCalculation(features) + case Gaussian => + features: Vector => gaussianCalculation(features) + } + } + + @Since("3.0.0") + override def predictRaw(features: Vector): Vector = predictRawFunc(features) + + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + var i = 0 + val size = dv.size + val maxLog = dv.values.max + while (i < size) { + dv.values(i) = math.exp(dv.values(i) - maxLog) + i += 1 + } + val probSum = dv.values.sum + i = 0 + while (i < size) { + dv.values(i) = dv.values(i) / probSum + i += 1 + } + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in NaiveBayesModel:" + + " raw2probabilityInPlace encountered SparseVector") + } + } + + @Since("1.5.0") + override def copy(extra: ParamMap): NaiveBayesModel = { + copyValues(new NaiveBayesModel(uid, pi, theta, sigma).setParent(this.parent), extra) + } + + @Since("1.5.0") + override def toString: String = { + s"NaiveBayesModel: uid=$uid, modelType=${$(modelType)}, numClasses=$numClasses, " + + s"numFeatures=$numFeatures" + } + + @Since("1.6.0") + override def write: MLWriter = new NaiveBayesModel.NaiveBayesModelWriter(this) +} + +@Since("1.6.0") +object NaiveBayesModel extends MLReadable[NaiveBayesModel] { + + @Since("1.6.0") + override def read: MLReader[NaiveBayesModel] = new NaiveBayesModelReader + + @Since("1.6.0") + override def load(path: String): NaiveBayesModel = super.load(path) + + /** [[MLWriter]] instance for [[NaiveBayesModel]] */ + private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { + import NaiveBayes._ + + private case class Data(pi: Vector, theta: Matrix, sigma: Matrix) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + val dataPath = new Path(path, "data").toString + + instance.getModelType match { + case Multinomial | Bernoulli | Complement => + require(instance.sigma.numRows == 0 && instance.sigma.numCols == 0) + case Gaussian => + require(instance.sigma.numRows != 0 && instance.sigma.numCols != 0) + } + + val data = Data(instance.pi, instance.theta, instance.sigma) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[NaiveBayesModel].getName + + override def load(path: String): NaiveBayesModel = { + implicit val format = DefaultFormats + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") + + val model = if (major.toInt < 3) { + val Row(pi: Vector, theta: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, Matrices.zeros(0, 0)) + } else { + val Row(pi: Vector, theta: Matrix, sigma: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta", "sigma") + .select("pi", "theta", "sigma") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, sigma) + } + + metadata.getAndSetParams(model) + model + } + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala new file mode 100644 index 000000000..2495d6496 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala @@ -0,0 +1,33 @@ +/* + * Copyright 2020 Intel Corporation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util.{Instrumentation, OneDAL} +import org.apache.spark.rdd.RDD + +class NaiveBayesImpl(val executorNum: Int, + val executorCores: Int + ) extends Serializable with Logging { + def train(features: RDD[Vector], labels: RDD[Double], + instr: Option[Instrumentation]): NaiveBayesModel = { + + OneDAL.vectorsToMergedNumericTables(features, executorNum) + null + } +} From c05893abb97ad25370d20298b8bc4ac0de1ec634 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 17:22:40 +0800 Subject: [PATCH 02/29] define ccl_root and ccl:gather --- mllib-dal/src/main/native/ALSDALImpl.cpp | 2 -- mllib-dal/src/main/native/KMeansDALImpl.cpp | 2 -- mllib-dal/src/main/native/OneCCL.cpp | 2 ++ mllib-dal/src/main/native/OneCCL.h | 26 +++++++++++++++++++ mllib-dal/src/main/native/PCADALImpl.cpp | 2 -- ...ayesImpl.scala => NaiveBayesDALImpl.scala} | 22 ++++++++++++++-- 6 files changed, 48 insertions(+), 8 deletions(-) rename mllib-dal/src/main/scala/org/apache/spark/ml/classification/{NaiveBayesImpl.scala => NaiveBayesDALImpl.scala} (54%) diff --git a/mllib-dal/src/main/native/ALSDALImpl.cpp b/mllib-dal/src/main/native/ALSDALImpl.cpp index 157b39bae..941b54c65 100644 --- a/mllib-dal/src/main/native/ALSDALImpl.cpp +++ b/mllib-dal/src/main/native/ALSDALImpl.cpp @@ -30,8 +30,6 @@ using namespace daal; using namespace daal::algorithms; using namespace daal::algorithms::implicit_als; -const int ccl_root = 0; - typedef float algorithmFPType; /* Algorithm floating-point type */ NumericTablePtr userOffset; diff --git a/mllib-dal/src/main/native/KMeansDALImpl.cpp b/mllib-dal/src/main/native/KMeansDALImpl.cpp index db8db80b1..3aa639336 100644 --- a/mllib-dal/src/main/native/KMeansDALImpl.cpp +++ b/mllib-dal/src/main/native/KMeansDALImpl.cpp @@ -27,8 +27,6 @@ using namespace std; using namespace daal; using namespace daal::algorithms; -const int ccl_root = 0; - typedef double algorithmFPType; /* Algorithm floating-point type */ static NumericTablePtr kmeans_compute(int rankId, ccl::communicator &comm, diff --git a/mllib-dal/src/main/native/OneCCL.cpp b/mllib-dal/src/main/native/OneCCL.cpp index 9b1aa6e2c..0565d8859 100644 --- a/mllib-dal/src/main/native/OneCCL.cpp +++ b/mllib-dal/src/main/native/OneCCL.cpp @@ -31,6 +31,8 @@ #include "OneCCL.h" #include "org_apache_spark_ml_util_OneCCL__.h" +extern const int ccl_root = 0; + static const int CCL_IP_LEN = 128; static std::list local_host_ips; static size_t comm_size = 0; diff --git a/mllib-dal/src/main/native/OneCCL.h b/mllib-dal/src/main/native/OneCCL.h index 056d898d3..9007f02c2 100644 --- a/mllib-dal/src/main/native/OneCCL.h +++ b/mllib-dal/src/main/native/OneCCL.h @@ -18,4 +18,30 @@ #include +#include +using namespace std; + +namespace ccl { + template (), event>::type> + event CCL_API gather(const BufferType *sendbuf, + int sendcount, + BufferType *recvbuf, + int recvcount, + const communicator& comm) { + auto comm_size = comm.size(); + vector send_counts(comm_size, 0); + vector recv_counts(comm_size, 0); + + const size_t root_rank = 0; + send_counts[root_rank] = sendcount; + + if (comm.rank() == root_rank) + std::fill(recv_counts.begin(), recv_counts.end(), sendcount); + + return ccl::alltoallv(sendbuf, send_counts, recvbuf, recv_counts, comm); + } +} + ccl::communicator &getComm(); +extern const int ccl_root; \ No newline at end of file diff --git a/mllib-dal/src/main/native/PCADALImpl.cpp b/mllib-dal/src/main/native/PCADALImpl.cpp index c04484a09..b829b2a83 100644 --- a/mllib-dal/src/main/native/PCADALImpl.cpp +++ b/mllib-dal/src/main/native/PCADALImpl.cpp @@ -26,8 +26,6 @@ using namespace std; using namespace daal; using namespace daal::algorithms; -const int ccl_root = 0; - typedef double algorithmFPType; /* Algorithm floating-point type */ /* diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala similarity index 54% rename from mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 2495d6496..58bfa4396 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -21,13 +21,31 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.util.{Instrumentation, OneDAL} import org.apache.spark.rdd.RDD -class NaiveBayesImpl(val executorNum: Int, +class NaiveBayesImpl(val classNum: Int, + val executorNum: Int, val executorCores: Int ) extends Serializable with Logging { def train(features: RDD[Vector], labels: RDD[Double], instr: Option[Instrumentation]): NaiveBayesModel = { - OneDAL.vectorsToMergedNumericTables(features, executorNum) + val featureTables = OneDAL.vectorsToMergedNumericTables(features, executorNum) + val labelTables = OneDAL.doublesToNumericTables(labels, executorNum) + + featureTables.zip(labelTables).mapPartitions { + case (tables: Iterator[(Long, Long)]) => + val (featureTabAddr, lableTabAddr) = tables.next() + + val result = new NaiveBayesResult + cNaiveBayesDALCompute(featureTabAddr, lableTabAddr, + classNum, executorNum, executorCores, result) + } + null } + + @native private def cNaiveBayesDALCompute(features: Long, labels: Long, + class_num: Int, + executor_num: Int, + executor_cores: Int, + result: NaiveBayesResult): Unit } From e4984aa66273e5c9e572d796f16bfc670072033b Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 17:26:04 +0800 Subject: [PATCH 03/29] Add NaiveBayesDALImpl scala,java & jni --- .../ml/classification/NaiveBayesResult.java | 4 + mllib-dal/src/main/native/Makefile | 6 +- .../src/main/native/NaiveBayesDALImpl.cpp | 92 +++++++++++++++++++ mllib-dal/src/main/native/build-jni.sh | 7 +- ...park_ml_classification_NaiveBayesDALImpl.h | 21 +++++ .../ml/classification/NaiveBayes.scala | 4 +- .../ml/classification/NaiveBayesDALImpl.scala | 8 +- 7 files changed, 133 insertions(+), 9 deletions(-) create mode 100644 mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java create mode 100644 mllib-dal/src/main/native/NaiveBayesDALImpl.cpp create mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_classification_NaiveBayesDALImpl.h diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java b/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java new file mode 100644 index 000000000..7eca76aee --- /dev/null +++ b/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java @@ -0,0 +1,4 @@ +package org.apache.spark.ml.classification; + +public class NaiveBayesResult { +} diff --git a/mllib-dal/src/main/native/Makefile b/mllib-dal/src/main/native/Makefile index 06b8e0c13..bf05e87d9 100644 --- a/mllib-dal/src/main/native/Makefile +++ b/mllib-dal/src/main/native/Makefile @@ -37,13 +37,15 @@ CPP_SRCS += \ ./OneCCL.cpp ./OneDAL.cpp ./service.cpp ./error_handling.cpp \ ./KMeansDALImpl.cpp \ ./PCADALImpl.cpp \ - ./ALSDALImpl.cpp ./ALSShuffle.cpp + ./ALSDALImpl.cpp ./ALSShuffle.cpp \ + ./NaiveBayesDALImpl.cpp OBJS += \ ./OneCCL.o ./OneDAL.o ./service.o ./error_handling.o \ ./KMeansDALImpl.o \ ./PCADALImpl.o \ - ./ALSDALImpl.o ./ALSShuffle.o + ./ALSDALImpl.o ./ALSShuffle.o \ + ./NaiveBayesDALImpl.o # Output Binary OUTPUT = ../../../target/libMLlibDAL.so diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp new file mode 100644 index 000000000..22cd9a518 --- /dev/null +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -0,0 +1,92 @@ +#include +#include "service.h" +#include "OneCCL.h" +#include "org_apache_spark_ml_classification_NaiveBayesDALImpl.h" + +using namespace std; +using namespace daal; +using namespace daal::algorithms; +using namespace daal::algorithms::multinomial_naive_bayes; + +/* + * Class: org_apache_spark_ml_classification_NaiveBayesDALImpl + * Method: cNaiveBayesDALCompute + * Signature: (JJIIILorg/apache/spark/ml/classification/NaiveBayesResult;)V + */ +JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute + (JNIEnv *env, jobject obj, jlong featuresTab, jlong labelsTab, + jint class_num, jint executor_num, jint executor_cores, jobject result) { + +} + +void trainModel(int nClasses, const ccl::communicator & comm) +{ + /* Initialize FileDataSource to retrieve the input data from a .csv file */ + FileDataSource trainDataSource("1", DataSource::doAllocateNumericTable, + DataSource::doDictionaryFromContext); + FileDataSource trainLabelsSource("2", DataSource::doAllocateNumericTable, + DataSource::doDictionaryFromContext); + + /* Retrieve the data from input files */ + trainDataSource.loadDataBlock(); + trainLabelsSource.loadDataBlock(); + + auto rankId = comm.rank(); + auto nBlocks = comm.size(); + + /* Create an algorithm object to train the Naive Bayes model based on the local-node data */ + training::Distributed localAlgorithm(nClasses); + + /* Pass a training data set and dependent values to the algorithm */ + localAlgorithm.input.set(classifier::training::data, trainDataSource.getNumericTable()); + localAlgorithm.input.set(classifier::training::labels, trainLabelsSource.getNumericTable()); + + /* Train the Naive Bayes model on local nodes */ + localAlgorithm.compute(); + + /* Serialize partial results required by step 2 */ + services::SharedPtr serializedData; + InputDataArchive dataArch; + localAlgorithm.getPartialResult()->serialize(dataArch); + size_t perNodeArchLength = dataArch.getSizeOfArchive(); + + /* Serialized data is of equal size on each node if each node called compute() equal number of times */ + if (rankId == ccl_root) + { + serializedData.reset(new byte[perNodeArchLength * nBlocks]); + } + + { + services::SharedPtr nodeResults(new byte[perNodeArchLength]); + dataArch.copyArchiveToArray(nodeResults.get(), perNodeArchLength); + + /* Transfer partial results to step 2 on the root node */ + // MPI_Gather(nodeResults.get(), perNodeArchLength, MPI_CHAR, serializedData.get(), perNodeArchLength, MPI_CHAR, mpi_root, MPI_COMM_WORLD); + ccl::gather(nodeResults.get(), perNodeArchLength, serializedData.get(), perNodeArchLength, comm).wait(); + } + + if (rankId == ccl_root) + { + /* Create an algorithm object to build the final Naive Bayes model on the master node */ + training::Distributed masterAlgorithm(nClasses); + + for (size_t i = 0; i < nBlocks; i++) + { + /* Deserialize partial results from step 1 */ + OutputDataArchive dataArch(serializedData.get() + perNodeArchLength * i, perNodeArchLength); + + training::PartialResultPtr dataForStep2FromStep1(new training::PartialResult()); + dataForStep2FromStep1->deserialize(dataArch); + + /* Set the local Naive Bayes model as input for the master-node algorithm */ + masterAlgorithm.input.add(training::partialModels, dataForStep2FromStep1); + } + + /* Merge and finalizeCompute the Naive Bayes model on the master node */ + masterAlgorithm.compute(); + masterAlgorithm.finalizeCompute(); + + /* Retrieve the algorithm results */ + training::ResultPtr trainingResult = masterAlgorithm.getResult(); + } +} \ No newline at end of file diff --git a/mllib-dal/src/main/native/build-jni.sh b/mllib-dal/src/main/native/build-jni.sh index 3a07d62fc..3f79bddf5 100755 --- a/mllib-dal/src/main/native/build-jni.sh +++ b/mllib-dal/src/main/native/build-jni.sh @@ -19,12 +19,12 @@ WORK_DIR="$( cd $( dirname "${BASH_SOURCE[0]}" ) && pwd )" DAAL_JAR=${ONEAPI_ROOT}/dal/latest/lib/onedal.jar if [ ! -f "$DAAL_JAR" ]; then - echo $DAAL_JAR does not exist! + echo \$DAAL_JAR does not exist! exit 1 fi if [[ ! -e "$SPARK_HOME" ]]; then - echo $SPARK_HOME does not exist! + echo \$SPARK_HOME does not exist! exit 1 fi @@ -33,4 +33,5 @@ javah -d $WORK_DIR/javah -classpath "$WORK_DIR/../../../target/classes:$DAAL_JAR org.apache.spark.ml.util.OneDAL$ \ org.apache.spark.ml.clustering.KMeansDALImpl \ org.apache.spark.ml.feature.PCADALImpl \ - org.apache.spark.ml.recommendation.ALSDALImpl + org.apache.spark.ml.recommendation.ALSDALImpl \ + org.apache.spark.ml.classification.NaiveBayesDALImpl \ diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_classification_NaiveBayesDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_classification_NaiveBayesDALImpl.h new file mode 100644 index 000000000..5c55d2e5b --- /dev/null +++ b/mllib-dal/src/main/native/javah/org_apache_spark_ml_classification_NaiveBayesDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class org_apache_spark_ml_classification_NaiveBayesDALImpl */ + +#ifndef _Included_org_apache_spark_ml_classification_NaiveBayesDALImpl +#define _Included_org_apache_spark_ml_classification_NaiveBayesDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: org_apache_spark_ml_classification_NaiveBayesDALImpl + * Method: cNaiveBayesDALCompute + * Signature: (JJIIILorg/apache/spark/ml/classification/NaiveBayesResult;)V + */ +JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute + (JNIEnv *, jobject, jlong, jlong, jint, jint, jint, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index b5828939f..1ebf4ff5e 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -183,6 +183,8 @@ class NaiveBayes @Since("1.5.0") ( logInfo(s"NaiveBayesDAL fit using $executor_num Executors") + val numClasses = getNumClasses(dataset) + val features: RDD[Vector] = dataset .select(DatasetUtils.columnToVector(dataset, getFeaturesCol)).rdd.map { case Row(feature: Vector) => feature @@ -191,7 +193,7 @@ class NaiveBayes @Since("1.5.0") ( case Row(label: Double) => label } - val model = new NaiveBayesImpl( + val model = new NaiveBayesDALImpl(numClasses, executor_num, executor_cores).train(features, labels, Some(instr)) model } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 58bfa4396..f16495631 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -21,9 +21,9 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.util.{Instrumentation, OneDAL} import org.apache.spark.rdd.RDD -class NaiveBayesImpl(val classNum: Int, - val executorNum: Int, - val executorCores: Int +class NaiveBayesDALImpl(val classNum: Int, + val executorNum: Int, + val executorCores: Int ) extends Serializable with Logging { def train(features: RDD[Vector], labels: RDD[Double], instr: Option[Instrumentation]): NaiveBayesModel = { @@ -38,6 +38,8 @@ class NaiveBayesImpl(val classNum: Int, val result = new NaiveBayesResult cNaiveBayesDALCompute(featureTabAddr, lableTabAddr, classNum, executorNum, executorCores, result) + + Iterator() } null From f4f2baaa0fd0fa4221cc0c47aa1d739b7018d273 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 17:26:27 +0800 Subject: [PATCH 04/29] Add doublesToNumericTables --- .../org/apache/spark/ml/util/OneDAL.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 62a803dc3..2def27f5e 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -89,6 +89,26 @@ object OneDAL { } } + def doublesToNumericTables(doubles: RDD[Double], executorNum: Int): RDD[Long] = { + require(executorNum > 0) + val doublesTables = doubles.repartition(executorNum).mapPartitions { it: Iterator[Double] => + val data = it.toArray + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrix = new DALMatrix(context, classOf[java.lang.Double], + 1, data.length, NumericTable.AllocationFlag.DoAllocate) + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + data.zipWithIndex.foreach { case (value: Double, index: Int) => + matrix.set(index, 0, value) + } + Iterator(matrix.getCNumericTable) + } + doublesTables + } + def vectorsToMergedNumericTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { require(executorNum > 0) From 47f9b95024df20bce15849c03bba6dc296718a19 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 17:43:14 +0800 Subject: [PATCH 05/29] Add trainModel and trainingResult --- .../src/main/native/NaiveBayesDALImpl.cpp | 69 +++++++++++-------- 1 file changed, 40 insertions(+), 29 deletions(-) diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index 22cd9a518..9700795f9 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -8,29 +8,11 @@ using namespace daal; using namespace daal::algorithms; using namespace daal::algorithms::multinomial_naive_bayes; -/* - * Class: org_apache_spark_ml_classification_NaiveBayesDALImpl - * Method: cNaiveBayesDALCompute - * Signature: (JJIIILorg/apache/spark/ml/classification/NaiveBayesResult;)V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute - (JNIEnv *env, jobject obj, jlong featuresTab, jlong labelsTab, - jint class_num, jint executor_num, jint executor_cores, jobject result) { - -} - -void trainModel(int nClasses, const ccl::communicator & comm) -{ - /* Initialize FileDataSource to retrieve the input data from a .csv file */ - FileDataSource trainDataSource("1", DataSource::doAllocateNumericTable, - DataSource::doDictionaryFromContext); - FileDataSource trainLabelsSource("2", DataSource::doAllocateNumericTable, - DataSource::doDictionaryFromContext); - - /* Retrieve the data from input files */ - trainDataSource.loadDataBlock(); - trainLabelsSource.loadDataBlock(); - +static training::ResultPtr trainModel(const ccl::communicator &comm, + const NumericTablePtr &featuresTab, + const NumericTablePtr &labelsTab, + int nClasses) +{ auto rankId = comm.rank(); auto nBlocks = comm.size(); @@ -38,14 +20,14 @@ void trainModel(int nClasses, const ccl::communicator & comm) training::Distributed localAlgorithm(nClasses); /* Pass a training data set and dependent values to the algorithm */ - localAlgorithm.input.set(classifier::training::data, trainDataSource.getNumericTable()); - localAlgorithm.input.set(classifier::training::labels, trainLabelsSource.getNumericTable()); + localAlgorithm.input.set(classifier::training::data, featuresTab); + localAlgorithm.input.set(classifier::training::labels, labelsTab); /* Train the Naive Bayes model on local nodes */ localAlgorithm.compute(); /* Serialize partial results required by step 2 */ - services::SharedPtr serializedData; + services::SharedPtr serializedData; InputDataArchive dataArch; localAlgorithm.getPartialResult()->serialize(dataArch); size_t perNodeArchLength = dataArch.getSizeOfArchive(); @@ -53,11 +35,11 @@ void trainModel(int nClasses, const ccl::communicator & comm) /* Serialized data is of equal size on each node if each node called compute() equal number of times */ if (rankId == ccl_root) { - serializedData.reset(new byte[perNodeArchLength * nBlocks]); + serializedData.reset(new daal::byte[perNodeArchLength * nBlocks]); } { - services::SharedPtr nodeResults(new byte[perNodeArchLength]); + services::SharedPtr nodeResults(new daal::byte[perNodeArchLength]); dataArch.copyArchiveToArray(nodeResults.get(), perNodeArchLength); /* Transfer partial results to step 2 on the root node */ @@ -88,5 +70,34 @@ void trainModel(int nClasses, const ccl::communicator & comm) /* Retrieve the algorithm results */ training::ResultPtr trainingResult = masterAlgorithm.getResult(); + return trainingResult; } -} \ No newline at end of file + return training::ResultPtr(); +} + +/* + * Class: org_apache_spark_ml_classification_NaiveBayesDALImpl + * Method: cNaiveBayesDALCompute + * Signature: (JJIIILorg/apache/spark/ml/classification/NaiveBayesResult;)V + */ +JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute + (JNIEnv *env, jobject obj, jlong pFeaturesTab, jlong pLabelsTab, + jint class_num, jint executor_num, jint executor_cores, jobject result) { + + ccl::communicator &comm = getComm(); + + NumericTablePtr featuresTab = *((NumericTablePtr *)pFeaturesTab); + NumericTablePtr labelsTab = *((NumericTablePtr *)pLabelsTab); + + // Set number of threads for oneDAL to use for each rank + services::Environment::getInstance()->setNumberOfThreads(executor_cores); + + int nThreadsNew = + services::Environment::getInstance()->getNumberOfThreads(); + cout << "oneDAL (native): Number of CPU threads used: " << nThreadsNew + << endl; + + training::ResultPtr trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); + + +} From e2c2a875cdefe325f9f8f1c61c7b97e308c2c677 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 18:12:20 +0800 Subject: [PATCH 06/29] Add getOneCCLIPPort to Utils --- .../main/scala/org/apache/spark/ml/util/Utils.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala index 525afc78b..775728aee 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala @@ -24,6 +24,17 @@ import org.apache.spark.rdd.RDD object Utils { + def getOneCCLIPPort(data: RDD[_]): String = { + val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) + val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", + executorIPAddress) + val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) + val kvsPort = data.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", + kvsPortDetected) + + kvsIP + "_" + kvsPort + } + // Return index -> (rows, cols) map def getPartitionDims(data: RDD[Vector]): Map[Int, (Int, Int)] = { var numCols: Int = 0 From c2fa328cd58d62c7e11e60a45cdb5661998a2c7e Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 11 May 2021 18:12:47 +0800 Subject: [PATCH 07/29] Add return model, to be filled --- .../ml/classification/NaiveBayes.scala | 2 +- .../ml/classification/NaiveBayesDALImpl.scala | 45 ++++++++++++++----- 2 files changed, 34 insertions(+), 13 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 1ebf4ff5e..3dc582543 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -193,7 +193,7 @@ class NaiveBayes @Since("1.5.0") ( case Row(label: Double) => label } - val model = new NaiveBayesDALImpl(numClasses, + val model = new NaiveBayesDALImpl(uid, numClasses, executor_num, executor_cores).train(features, labels, Some(instr)) model } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index f16495631..5f502180b 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -17,32 +17,53 @@ package org.apache.spark.ml.classification import org.apache.spark.internal.Logging -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.util.{Instrumentation, OneDAL} +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.ml.util.Utils.getOneCCLIPPort +import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} import org.apache.spark.rdd.RDD -class NaiveBayesDALImpl(val classNum: Int, +class NaiveBayesDALImpl(val uid: String, + val classNum: Int, val executorNum: Int, val executorCores: Int ) extends Serializable with Logging { def train(features: RDD[Vector], labels: RDD[Double], instr: Option[Instrumentation]): NaiveBayesModel = { + val kvsIPPort = getOneCCLIPPort(features) + val featureTables = OneDAL.vectorsToMergedNumericTables(features, executorNum) val labelTables = OneDAL.doublesToNumericTables(labels, executorNum) - featureTables.zip(labelTables).mapPartitions { - case (tables: Iterator[(Long, Long)]) => - val (featureTabAddr, lableTabAddr) = tables.next() + val results = featureTables.zip(labelTables).mapPartitionsWithIndex { + case (rank: Int, tables: Iterator[(Long, Long)]) => + val (featureTabAddr, lableTabAddr) = tables.next() + + OneCCL.init(executorNum, rank, kvsIPPort) + + val result = new NaiveBayesResult + cNaiveBayesDALCompute(featureTabAddr, lableTabAddr, + classNum, executorNum, executorCores, result) + + val ret = if (OneCCL.isRoot()) { + Iterator(result) + } else { + Iterator.empty + } + + OneCCL.cleanup() + ret + }.collect() - val result = new NaiveBayesResult - cNaiveBayesDALCompute(featureTabAddr, lableTabAddr, - classNum, executorNum, executorCores, result) + // Make sure there is only one result from rank 0 + assert(results.length == 1) - Iterator() - } + val model = new NaiveBayesModel(uid, + Vectors.zeros(classNum), + Matrices.zeros(0, 0), + Matrices.zeros(0, 0)) + model - null } @native private def cNaiveBayesDALCompute(features: Long, labels: Long, From 2a0624f8ea7bae3bc93700aab971b71b32e36016 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 12 May 2021 09:29:02 +0800 Subject: [PATCH 08/29] Fix format-cpp --- dev/codestyle/format-cpp.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/codestyle/format-cpp.sh b/dev/codestyle/format-cpp.sh index f81331202..719d4689b 100755 --- a/dev/codestyle/format-cpp.sh +++ b/dev/codestyle/format-cpp.sh @@ -21,7 +21,7 @@ if [ -z $CLANG_FORMAT ]; then exit 1 fi -if [ -f .clang-format ]; then +if [ ! -f .clang-format ]; then echo .clang-format is not found in current directory, please generate it. exit 1 fi From 26e6d5b6cbf51e9012267188dbeb7b45b8c8ea50 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 12 May 2021 14:50:38 +0800 Subject: [PATCH 09/29] Add numericTableNx1ToVector & numericTableToMatrix --- .../org/apache/spark/ml/util/OneDAL.scala | 28 +++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 2def27f5e..672d8596d 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -17,13 +17,10 @@ package org.apache.spark.ml.util import java.util.logging.{Level, Logger} - -import com.intel.daal.data_management.data.{HomogenNumericTable, Matrix => DALMatrix, NumericTable, - RowMergedNumericTable} +import com.intel.daal.data_management.data.{HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext - import org.apache.spark.SparkContext -import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} import org.apache.spark.storage.StorageLevel @@ -33,6 +30,27 @@ object OneDAL { private val logger = Logger.getLogger("util.OneDAL") private val logLevel = Level.INFO + def numericTableToMatrix(table: NumericTable): Matrix = { + val numRows = table.getNumberOfRows.toInt + val numCols = table.getNumberOfColumns.toInt + val matrix = Matrices.zeros(numRows, numCols) + for (row <- 0 until numRows) { + for (col <- 0 until numCols) { + matrix.update(row, col, table.getDoubleValue(col, row)) + } + } + matrix + } + + def numericTableNx1ToVector(table: NumericTable): Vector = { + val numRows = table.getNumberOfRows.toInt + val internArray = new Array[Double](numRows.toInt) + for (row <- 0 until numRows) { + internArray(row) = table.getDoubleValue(0, row) + } + Vectors.dense(internArray) + } + // Convert DAL numeric table to array of vectors def numericTableToVectors(table: NumericTable): Array[Vector] = { val numRows = table.getNumberOfRows.toInt From ef101c00477df4899d07a3b8f50beafef5e421f2 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 12 May 2021 14:50:56 +0800 Subject: [PATCH 10/29] format code --- mllib-dal/src/main/native/OneCCL.h | 39 +++++++++++++++--------------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/mllib-dal/src/main/native/OneCCL.h b/mllib-dal/src/main/native/OneCCL.h index 9007f02c2..5550ce904 100644 --- a/mllib-dal/src/main/native/OneCCL.h +++ b/mllib-dal/src/main/native/OneCCL.h @@ -22,26 +22,25 @@ using namespace std; namespace ccl { - template (), event>::type> - event CCL_API gather(const BufferType *sendbuf, - int sendcount, - BufferType *recvbuf, - int recvcount, - const communicator& comm) { - auto comm_size = comm.size(); - vector send_counts(comm_size, 0); - vector recv_counts(comm_size, 0); - - const size_t root_rank = 0; - send_counts[root_rank] = sendcount; - - if (comm.rank() == root_rank) - std::fill(recv_counts.begin(), recv_counts.end(), sendcount); - - return ccl::alltoallv(sendbuf, send_counts, recvbuf, recv_counts, comm); - } +template (), event>::type> +event CCL_API gather(const BufferType *sendbuf, int sendcount, + BufferType *recvbuf, int recvcount, + const communicator &comm) { + auto comm_size = comm.size(); + vector send_counts(comm_size, 0); + vector recv_counts(comm_size, 0); + + const size_t root_rank = 0; + send_counts[root_rank] = sendcount; + + if (comm.rank() == root_rank) + std::fill(recv_counts.begin(), recv_counts.end(), sendcount); + + return ccl::alltoallv(sendbuf, send_counts, recvbuf, recv_counts, comm); } +} // namespace ccl ccl::communicator &getComm(); -extern const int ccl_root; \ No newline at end of file +extern const int ccl_root; From 8fc9a4f435aa0e893133251fefab143f85e5bc32 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 12 May 2021 14:51:18 +0800 Subject: [PATCH 11/29] CSR support --- .../ml/classification/NaiveBayesResult.java | 2 + .../src/main/native/NaiveBayesDALImpl.cpp | 104 +++++++++++++----- .../ml/classification/NaiveBayesDALImpl.scala | 6 +- 3 files changed, 79 insertions(+), 33 deletions(-) diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java b/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java index 7eca76aee..bf5e51941 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java +++ b/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java @@ -1,4 +1,6 @@ package org.apache.spark.ml.classification; public class NaiveBayesResult { + public long piNumericTable; + public long thetaNumericTable; } diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index 9700795f9..569afba8b 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -1,23 +1,28 @@ #include -#include "service.h" + #include "OneCCL.h" #include "org_apache_spark_ml_classification_NaiveBayesDALImpl.h" +#include "service.h" using namespace std; using namespace daal; using namespace daal::algorithms; +using namespace daal::data_management; using namespace daal::algorithms::multinomial_naive_bayes; -static training::ResultPtr trainModel(const ccl::communicator &comm, - const NumericTablePtr &featuresTab, - const NumericTablePtr &labelsTab, - int nClasses) -{ +typedef double algorithmFPType; /* Algorithm floating-point type */ + +template +static training::ResultPtr +trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, + const NumericTablePtr &labelsTab, int nClasses) { auto rankId = comm.rank(); auto nBlocks = comm.size(); - /* Create an algorithm object to train the Naive Bayes model based on the local-node data */ - training::Distributed localAlgorithm(nClasses); + /* Create an algorithm object to train the Naive Bayes model based on the + * local-node data */ + training::Distributed localAlgorithm( + nClasses); /* Pass a training data set and dependent values to the algorithm */ localAlgorithm.input.set(classifier::training::data, featuresTab); @@ -32,36 +37,46 @@ static training::ResultPtr trainModel(const ccl::communicator &comm, localAlgorithm.getPartialResult()->serialize(dataArch); size_t perNodeArchLength = dataArch.getSizeOfArchive(); - /* Serialized data is of equal size on each node if each node called compute() equal number of times */ - if (rankId == ccl_root) - { + /* Serialized data is of equal size on each node if each node called + * compute() equal number of times */ + if (rankId == ccl_root) { serializedData.reset(new daal::byte[perNodeArchLength * nBlocks]); } { - services::SharedPtr nodeResults(new daal::byte[perNodeArchLength]); + services::SharedPtr nodeResults( + new daal::byte[perNodeArchLength]); dataArch.copyArchiveToArray(nodeResults.get(), perNodeArchLength); /* Transfer partial results to step 2 on the root node */ - // MPI_Gather(nodeResults.get(), perNodeArchLength, MPI_CHAR, serializedData.get(), perNodeArchLength, MPI_CHAR, mpi_root, MPI_COMM_WORLD); - ccl::gather(nodeResults.get(), perNodeArchLength, serializedData.get(), perNodeArchLength, comm).wait(); + // MPI_Gather(nodeResults.get(), perNodeArchLength, MPI_CHAR, + // serializedData.get(), perNodeArchLength, MPI_CHAR, mpi_root, + // MPI_COMM_WORLD); + ccl::gather(nodeResults.get(), perNodeArchLength, serializedData.get(), + perNodeArchLength, comm) + .wait(); } - if (rankId == ccl_root) - { - /* Create an algorithm object to build the final Naive Bayes model on the master node */ - training::Distributed masterAlgorithm(nClasses); + if (rankId == ccl_root) { + /* Create an algorithm object to build the final Naive Bayes model on + * the master node */ + training::Distributed + masterAlgorithm(nClasses); - for (size_t i = 0; i < nBlocks; i++) - { + for (size_t i = 0; i < nBlocks; i++) { /* Deserialize partial results from step 1 */ - OutputDataArchive dataArch(serializedData.get() + perNodeArchLength * i, perNodeArchLength); + OutputDataArchive dataArch(serializedData.get() + + perNodeArchLength * i, + perNodeArchLength); - training::PartialResultPtr dataForStep2FromStep1(new training::PartialResult()); + training::PartialResultPtr dataForStep2FromStep1( + new training::PartialResult()); dataForStep2FromStep1->deserialize(dataArch); - /* Set the local Naive Bayes model as input for the master-node algorithm */ - masterAlgorithm.input.add(training::partialModels, dataForStep2FromStep1); + /* Set the local Naive Bayes model as input for the master-node + * algorithm */ + masterAlgorithm.input.add(training::partialModels, + dataForStep2FromStep1); } /* Merge and finalizeCompute the Naive Bayes model on the master node */ @@ -70,6 +85,7 @@ static training::ResultPtr trainModel(const ccl::communicator &comm, /* Retrieve the algorithm results */ training::ResultPtr trainingResult = masterAlgorithm.getResult(); + return trainingResult; } return training::ResultPtr(); @@ -80,10 +96,11 @@ static training::ResultPtr trainModel(const ccl::communicator &comm, * Method: cNaiveBayesDALCompute * Signature: (JJIIILorg/apache/spark/ml/classification/NaiveBayesResult;)V */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute - (JNIEnv *env, jobject obj, jlong pFeaturesTab, jlong pLabelsTab, - jint class_num, jint executor_num, jint executor_cores, jobject result) { - +JNIEXPORT void JNICALL +Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( + JNIEnv *env, jobject obj, jlong pFeaturesTab, jlong pLabelsTab, + jint class_num, jint executor_num, jint executor_cores, jobject resultObj) { + ccl::communicator &comm = getComm(); NumericTablePtr featuresTab = *((NumericTablePtr *)pFeaturesTab); @@ -97,7 +114,34 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl cout << "oneDAL (native): Number of CPU threads used: " << nThreadsNew << endl; - training::ResultPtr trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); - + // Support both dense and csr numeric table + training::ResultPtr trainingResult; + if (featuresTab->getDataLayout() == NumericTable::StorageLayout::csrArray) { + trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); + } else { + trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); + } + + multinomial_naive_bayes::ModelPtr model = + trainingResult->get(classifier::training::model); + auto pi = model->getLogP(); + auto theta = model->getLogTheta(); + + printNumericTable(pi, "log of class priors"); + printNumericTable(theta, "log of class conditional probabilities"); + + // Return all log of class priors (LogP) and log of class conditional + // probabilities (LogTheta) + + // Get the class of the input object + jclass clazz = env->GetObjectClass(resultObj); + // Get Field references + jfieldID piNumericTableField = + env->GetFieldID(clazz, "piNumericTable", "J"); + jfieldID thetaNumericTableField = + env->GetFieldID(clazz, "thetaNumericTable", "J"); + + env->SetLongField(resultObj, piNumericTableField, (jlong)pi); + env->SetLongField(resultObj, thetaNumericTableField, (jlong)theta); } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 5f502180b..e1830df20 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -57,13 +57,13 @@ class NaiveBayesDALImpl(val uid: String, // Make sure there is only one result from rank 0 assert(results.length == 1) + val result = results(0) val model = new NaiveBayesModel(uid, - Vectors.zeros(classNum), - Matrices.zeros(0, 0), + OneDAL.numericTableNx1ToVector(OneDAL.makeNumericTable(result.piNumericTable)), + OneDAL.numericTableToMatrix(OneDAL.makeNumericTable(result.thetaNumericTable)), Matrices.zeros(0, 0)) model - } @native private def cNaiveBayesDALCompute(features: Long, labels: Long, From 4cffed4994df1f6f7112705534e9a0279e8a2e92 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 12 May 2021 19:27:26 +0800 Subject: [PATCH 12/29] Add labeledPointsToMergedNumericTables, to be tested --- mllib-dal/src/main/native/OneDAL.cpp | 14 ++++++ .../javah/org_apache_spark_ml_util_OneDAL__.h | 8 ++++ .../ml/classification/NaiveBayes.scala | 11 ++--- .../ml/classification/NaiveBayesDALImpl.scala | 13 ++++-- .../org/apache/spark/ml/util/OneDAL.scala | 46 ++++++++++++++++++- 5 files changed, 78 insertions(+), 14 deletions(-) diff --git a/mllib-dal/src/main/native/OneDAL.cpp b/mllib-dal/src/main/native/OneDAL.cpp index 9c3c48657..464078d58 100644 --- a/mllib-dal/src/main/native/OneDAL.cpp +++ b/mllib-dal/src/main/native/OneDAL.cpp @@ -42,6 +42,20 @@ Java_org_apache_spark_ml_util_OneDAL_00024_setNumericTableValue( (*nt)[row][column] = (double)value; } +/* + * Class: org_apache_spark_ml_util_OneDAL__ + * Method: cSetDouble + * Signature: (JIID)V + */ +JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDouble + (JNIEnv *env, jobject, jlong numTableAddr, jint row, jint column, jdouble value) { + HomogenNumericTable *nt = + static_cast *>( + ((SerializationIfacePtr *)numTableAddr)->get()); + (*nt)[row][column] = (double)value; + +} + /* * Class: org_apache_spark_ml_util_OneDAL__ * Method: cSetDoubleBatch diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h index d88462d66..3e862c3db 100644 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h +++ b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h @@ -23,6 +23,14 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_setNumericTabl JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cAddNumericTable (JNIEnv *, jobject, jlong, jlong); +/* + * Class: org_apache_spark_ml_util_OneDAL__ + * Method: cSetDouble + * Signature: (JIID)V + */ +JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDouble + (JNIEnv *, jobject, jlong, jint, jint, jdouble); + /* * Class: org_apache_spark_ml_util_OneDAL__ * Method: cSetDoubleBatch diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 3dc582543..35b2882f5 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -185,16 +185,13 @@ class NaiveBayes @Since("1.5.0") ( val numClasses = getNumClasses(dataset) - val features: RDD[Vector] = dataset - .select(DatasetUtils.columnToVector(dataset, getFeaturesCol)).rdd.map { - case Row(feature: Vector) => feature - } - val labels: RDD[Double] = dataset.select(col(getLabelCol)).rdd.map { - case Row(label: Double) => label + val labeledPoints: RDD[(Vector, Double)] = dataset + .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), col(getLabelCol)).rdd.map { + case Row(feature: Vector, label: Double) => (feature, label) } val model = new NaiveBayesDALImpl(uid, numClasses, - executor_num, executor_cores).train(features, labels, Some(instr)) + executor_num, executor_cores).train(labeledPoints, Some(instr)) model } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index e1830df20..04df176e7 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.classification import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.ml.util.Utils.getOneCCLIPPort import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} @@ -27,15 +28,17 @@ class NaiveBayesDALImpl(val uid: String, val executorNum: Int, val executorCores: Int ) extends Serializable with Logging { - def train(features: RDD[Vector], labels: RDD[Double], + def train(labeledPoints: RDD[(Vector, Double)], instr: Option[Instrumentation]): NaiveBayesModel = { - val kvsIPPort = getOneCCLIPPort(features) + val kvsIPPort = getOneCCLIPPort(labeledPoints) - val featureTables = OneDAL.vectorsToMergedNumericTables(features, executorNum) - val labelTables = OneDAL.doublesToNumericTables(labels, executorNum) +// val featureTables = OneDAL.vectorsToMergedNumericTables(features, executorNum) +// val labelTables = OneDAL.doublesToNumericTables(labels, executorNum) - val results = featureTables.zip(labelTables).mapPartitionsWithIndex { + val labeledPointsTables = OneDAL.labeledPointsToMergedNumericTables(labeledPoints, executorNum) + + val results = labeledPointsTables.mapPartitionsWithIndex { case (rank: Int, tables: Iterator[(Long, Long)]) => val (featureTabAddr, lableTabAddr) = tables.next() diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 672d8596d..252b18904 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.util import java.util.logging.{Level, Logger} -import com.intel.daal.data_management.data.{HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} +import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext import org.apache.spark.SparkContext import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} @@ -69,6 +69,12 @@ object OneDAL { resArray } +// def vectorsToCSRNumericTable(vectors: Iterator[Vector]): CSRNumericTable = { +// vectors.foreach { v => +// +// } +// } + def makeNumericTable(cData: Long): NumericTable = { val context = new DaalContext() @@ -120,13 +126,47 @@ object OneDAL { LibLoader.loadLibraries() data.zipWithIndex.foreach { case (value: Double, index: Int) => - matrix.set(index, 0, value) + cSetDouble(matrix.getCNumericTable, index, 0, value) } Iterator(matrix.getCNumericTable) } + doublesTables.count() + doublesTables } + def labeledPointsToMergedNumericTables(labeledPoints: RDD[(Vector, Double)], + executorNum: Int): RDD[(Long, Long)] = { + require(executorNum > 0) + + logger.info(s"Processing partitions with $executorNum executors") + + val tables = labeledPoints.repartition(executorNum).mapPartitions { + it: Iterator[(Vector, Double)] => + val points: Array[(Vector, Double)] = it.toArray + val numColumns = points(0)._1.size + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrixFeature = new DALMatrix(context, classOf[java.lang.Double], + numColumns, points.length, NumericTable.AllocationFlag.DoAllocate) + val matrixLabel = new DALMatrix(context, classOf[java.lang.Double], + 1, points.length, NumericTable.AllocationFlag.DoAllocate) + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + points.zipWithIndex.foreach { case (point: (Vector, Double), index: Int) => + val rowArray = point._1.toArray + cSetDoubleBatch(matrixFeature.getCNumericTable, index, rowArray, 1, numColumns) + cSetDouble(matrixLabel.getCNumericTable, index, 0, point._2) + } + Iterator((matrixFeature.getCNumericTable, matrixLabel.getCNumericTable)) + } + tables.count() + + tables + } + def vectorsToMergedNumericTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { require(executorNum > 0) @@ -207,6 +247,8 @@ object OneDAL { @native def cAddNumericTable(cObject: Long, numericTableAddr: Long) + @native def cSetDouble(numTableAddr: Long, row: Int, column: Int, value: Double) + @native def cSetDoubleBatch(numTableAddr: Long, curRows: Int, batch: Array[Double], numRows: Int, numCols: Int) From ea02438c4999b033cc3181360a2a9e30b477c023 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 13 May 2021 10:54:16 +0800 Subject: [PATCH 13/29] format code --- .../org/apache/spark/ml/util/OneDAL.scala | 57 +++++++++++-------- 1 file changed, 32 insertions(+), 25 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 252b18904..b2e06c696 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -16,8 +16,7 @@ package org.apache.spark.ml.util -import java.util.logging.{Level, Logger} -import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} +import com.intel.daal.data_management.data.{HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext import org.apache.spark.SparkContext import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} @@ -25,6 +24,8 @@ import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} import org.apache.spark.storage.StorageLevel +import java.util.logging.{Level, Logger} + object OneDAL { private val logger = Logger.getLogger("util.OneDAL") @@ -69,11 +70,11 @@ object OneDAL { resArray } -// def vectorsToCSRNumericTable(vectors: Iterator[Vector]): CSRNumericTable = { -// vectors.foreach { v => -// -// } -// } + // def vectorsToCSRNumericTable(vectors: Iterator[Vector]): CSRNumericTable = { + // vectors.foreach { v => + // + // } + // } def makeNumericTable(cData: Long): NumericTable = { @@ -136,31 +137,37 @@ object OneDAL { } def labeledPointsToMergedNumericTables(labeledPoints: RDD[(Vector, Double)], - executorNum: Int): RDD[(Long, Long)] = { + executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) logger.info(s"Processing partitions with $executorNum executors") val tables = labeledPoints.repartition(executorNum).mapPartitions { it: Iterator[(Vector, Double)] => - val points: Array[(Vector, Double)] = it.toArray - val numColumns = points(0)._1.size - // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc - val context = new DaalContext() - val matrixFeature = new DALMatrix(context, classOf[java.lang.Double], - numColumns, points.length, NumericTable.AllocationFlag.DoAllocate) - val matrixLabel = new DALMatrix(context, classOf[java.lang.Double], - 1, points.length, NumericTable.AllocationFlag.DoAllocate) - // oneDAL libs should be loaded by now, loading other native libs - logger.info("Loading native libraries") - LibLoader.loadLibraries() + val points: Array[(Vector, Double)] = it.toArray + val numColumns = points(0)._1.size + + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrixFeature = new DALMatrix(context, classOf[java.lang.Double], + numColumns, points.length, NumericTable.AllocationFlag.DoAllocate) + val matrixLabel = new DALMatrix(context, classOf[java.lang.Double], + 1, points.length, NumericTable.AllocationFlag.DoAllocate) + + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + points.zipWithIndex.foreach { case (point: (Vector, Double), index: Int) => + val rowArray = point._1.toArray + cSetDoubleBatch(matrixFeature.getCNumericTable, index, rowArray, 1, numColumns) + cSetDouble(matrixLabel.getCNumericTable, index, 0, point._2) + } - points.zipWithIndex.foreach { case (point: (Vector, Double), index: Int) => - val rowArray = point._1.toArray - cSetDoubleBatch(matrixFeature.getCNumericTable, index, rowArray, 1, numColumns) - cSetDouble(matrixLabel.getCNumericTable, index, 0, point._2) - } - Iterator((matrixFeature.getCNumericTable, matrixLabel.getCNumericTable)) + println(matrixFeature.getNumberOfRows, matrixFeature.getNumberOfColumns) + println(matrixLabel.getNumberOfRows, matrixLabel.getNumberOfColumns) + + Iterator((matrixFeature.getCNumericTable, matrixLabel.getCNumericTable)) } tables.count() From 8098c4fc47fa3c0e64c30764d6157738010ec254 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 13 May 2021 16:33:49 +0800 Subject: [PATCH 14/29] Fix result return bug --- .../src/main/native/NaiveBayesDALImpl.cpp | 56 +++++++++++-------- .../ml/classification/NaiveBayesDALImpl.scala | 10 ++-- 2 files changed, 39 insertions(+), 27 deletions(-) diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index 569afba8b..77df54b02 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -100,8 +100,9 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( JNIEnv *env, jobject obj, jlong pFeaturesTab, jlong pLabelsTab, jint class_num, jint executor_num, jint executor_cores, jobject resultObj) { - - ccl::communicator &comm = getComm(); + + ccl::communicator &comm = getComm(); + auto rankId = comm.rank(); NumericTablePtr featuresTab = *((NumericTablePtr *)pFeaturesTab); NumericTablePtr labelsTab = *((NumericTablePtr *)pLabelsTab); @@ -114,7 +115,6 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( cout << "oneDAL (native): Number of CPU threads used: " << nThreadsNew << endl; - // Support both dense and csr numeric table training::ResultPtr trainingResult; if (featuresTab->getDataLayout() == NumericTable::StorageLayout::csrArray) { @@ -123,25 +123,35 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); } - multinomial_naive_bayes::ModelPtr model = - trainingResult->get(classifier::training::model); - auto pi = model->getLogP(); - auto theta = model->getLogTheta(); - - printNumericTable(pi, "log of class priors"); - printNumericTable(theta, "log of class conditional probabilities"); + cout << "oneDAL (native): training model finished" << endl; - // Return all log of class priors (LogP) and log of class conditional - // probabilities (LogTheta) - - // Get the class of the input object - jclass clazz = env->GetObjectClass(resultObj); - // Get Field references - jfieldID piNumericTableField = - env->GetFieldID(clazz, "piNumericTable", "J"); - jfieldID thetaNumericTableField = - env->GetFieldID(clazz, "thetaNumericTable", "J"); - - env->SetLongField(resultObj, piNumericTableField, (jlong)pi); - env->SetLongField(resultObj, thetaNumericTableField, (jlong)theta); + if (rankId == ccl_root) { + multinomial_naive_bayes::ModelPtr model = + trainingResult->get(classifier::training::model); + + // auto pi = model->getLogP(); + // auto theta = model->getLogTheta(); + + // printNumericTable(pi, "log of class priors", 10, 20); + // printNumericTable(theta, "log of class conditional probabilities", 10, 20); + + // Return all log of class priors (LogP) and log of class conditional + // probabilities (LogTheta) + + // Get the class of the input object + jclass clazz = env->GetObjectClass(resultObj); + // Get Field references + jfieldID piNumericTableField = + env->GetFieldID(clazz, "piNumericTable", "J"); + jfieldID thetaNumericTableField = + env->GetFieldID(clazz, "thetaNumericTable", "J"); + + NumericTablePtr *pi = + new NumericTablePtr(model->getLogP()); + NumericTablePtr *theta = + new NumericTablePtr(model->getLogTheta()); + + env->SetLongField(resultObj, piNumericTableField, (jlong)pi); + env->SetLongField(resultObj, thetaNumericTableField, (jlong)theta); + } } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 04df176e7..0238ef1b1 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -20,7 +20,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} +import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL, Service} import org.apache.spark.rdd.RDD class NaiveBayesDALImpl(val uid: String, @@ -49,7 +49,9 @@ class NaiveBayesDALImpl(val uid: String, classNum, executorNum, executorCores, result) val ret = if (OneCCL.isRoot()) { - Iterator(result) + val pi = OneDAL.numericTableNx1ToVector(OneDAL.makeNumericTable(result.piNumericTable)) + val theta = OneDAL.numericTableToMatrix(OneDAL.makeNumericTable(result.thetaNumericTable)) + Iterator((pi, theta)) } else { Iterator.empty } @@ -63,8 +65,8 @@ class NaiveBayesDALImpl(val uid: String, val result = results(0) val model = new NaiveBayesModel(uid, - OneDAL.numericTableNx1ToVector(OneDAL.makeNumericTable(result.piNumericTable)), - OneDAL.numericTableToMatrix(OneDAL.makeNumericTable(result.thetaNumericTable)), + result._1, + result._2, Matrices.zeros(0, 0)) model } From bfa71358d7c1d8174b816c336e5d7ab26243aa79 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 13 May 2021 16:34:25 +0800 Subject: [PATCH 15/29] Add oldLabels to be compatible with mllib bayes --- .../spark-3.0.0/ml/classification/NaiveBayes.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 35b2882f5..7bd60e5cc 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -34,6 +34,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.util.VersionUtils +import scala.sys.exit + /** * Params for Naive Bayes Classifiers. */ @@ -159,7 +161,8 @@ class NaiveBayes @Since("1.5.0") ( val isPlatformSupported = Utils.checkClusterPlatformCompatibility( dataset.sparkSession.sparkContext) val handleWeight = (isDefined(weightCol) && $(weightCol).nonEmpty) - if (isPlatformSupported && !handleWeight) { + val handleSmoothing = ($(smoothing) != 1.0) + if (isPlatformSupported && !handleWeight && !handleSmoothing) { trainNaiveBayesDAL(dataset, instr) } else { trainDiscreteImpl(dataset, instr) @@ -192,6 +195,12 @@ class NaiveBayes @Since("1.5.0") ( val model = new NaiveBayesDALImpl(uid, numClasses, executor_num, executor_cores).train(labeledPoints, Some(instr)) + + // Set labels to be compatible with old mllib model + val labels = (0 until numClasses).map(_.toDouble).toArray + + model.setOldLabels(labels) + model } From 057af90ed42caff1a6add8ddb1b56dfefc958ee6 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Mon, 17 May 2021 16:12:16 +0800 Subject: [PATCH 16/29] Refactor and support convert to CSR table --- .../src/main/native/NaiveBayesDALImpl.cpp | 2 + mllib-dal/src/main/native/OneDAL.cpp | 62 +++++-- .../javah/org_apache_spark_ml_util_OneDAL__.h | 20 +-- .../ml/classification/NaiveBayes.scala | 18 +- .../ml/classification/NaiveBayesDALImpl.scala | 2 +- .../spark/ml/clustering/KMeansDALImpl.scala | 2 +- .../apache/spark/ml/feature/PCADALImpl.scala | 2 +- .../spark/ml/recommendation/ALSDALImpl.scala | 2 +- .../org/apache/spark/ml/util/OneDAL.scala | 165 ++++++++++++------ 9 files changed, 184 insertions(+), 91 deletions(-) diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index 77df54b02..f7642f1ae 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -118,8 +118,10 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( // Support both dense and csr numeric table training::ResultPtr trainingResult; if (featuresTab->getDataLayout() == NumericTable::StorageLayout::csrArray) { + cout << "oneDAL (native): training model with fastCSR method" << endl; trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); } else { + cout << "oneDAL (native): training model with defaultDense method" << endl; trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); } diff --git a/mllib-dal/src/main/native/OneDAL.cpp b/mllib-dal/src/main/native/OneDAL.cpp index 464078d58..992fcef2f 100644 --- a/mllib-dal/src/main/native/OneDAL.cpp +++ b/mllib-dal/src/main/native/OneDAL.cpp @@ -27,21 +27,6 @@ using namespace daal::data_management; // Use oneDAL lib function extern bool daal_check_is_intel_cpu(); -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: setNumericTableValue - * Signature: (JIID)V - */ -JNIEXPORT void JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_setNumericTableValue( - JNIEnv *, jobject, jlong numTableAddr, jint row, jint column, - jdouble value) { - HomogenNumericTable *nt = - static_cast *>( - ((SerializationIfacePtr *)numTableAddr)->get()); - (*nt)[row][column] = (double)value; -} - /* * Class: org_apache_spark_ml_util_OneDAL__ * Method: cSetDouble @@ -117,11 +102,11 @@ Java_org_apache_spark_ml_util_OneDAL_00024_cCheckPlatformCompatibility( /* * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cNewCSRNumericTable + * Method: cNewCSRNumericTableFloat * Signature: ([F[J[JJJ)J */ JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTable( +Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableFloat( JNIEnv *env, jobject, jfloatArray data, jlongArray colIndices, jlongArray rowOffsets, jlong nFeatures, jlong nVectors) { @@ -157,3 +142,46 @@ Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTable( return (jlong)ret; } + +/* + * Class: org_apache_spark_ml_util_OneDAL__ + * Method: cNewCSRNumericTableDouble + * Signature: ([D[J[JJJ)J + */ +JNIEXPORT jlong JNICALL +Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableDouble( + JNIEnv *env, jobject, jdoubleArray data, jlongArray colIndices, + jlongArray rowOffsets, jlong nFeatures, jlong nVectors) { + + long numData = env->GetArrayLength(data); + + size_t *resultRowOffsets = NULL; + size_t *resultColIndices = NULL; + double *resultData = NULL; + + CSRNumericTable *numericTable = new CSRNumericTable( + resultData, resultColIndices, resultRowOffsets, nFeatures, nVectors); + numericTable->allocateDataMemory(numData); + numericTable->getArrays(&resultData, &resultColIndices, + &resultRowOffsets); + + size_t *pRowOffsets = (size_t *)env->GetLongArrayElements(rowOffsets, 0); + size_t *pColIndices = (size_t *)env->GetLongArrayElements(colIndices, 0); + double *pData = env->GetDoubleArrayElements(data, 0); + + for (size_t i = 0; i < (size_t)numData; ++i) { + resultData[i] = pData[i]; + resultColIndices[i] = pColIndices[i]; + } + for (size_t i = 0; i < (size_t)nVectors + 1; ++i) { + resultRowOffsets[i] = pRowOffsets[i]; + } + + env->ReleaseLongArrayElements(rowOffsets, (jlong *)pRowOffsets, 0); + env->ReleaseLongArrayElements(colIndices, (jlong *)pColIndices, 0); + env->ReleaseDoubleArrayElements(data, pData, 0); + + CSRNumericTablePtr *ret = new CSRNumericTablePtr(numericTable); + + return (jlong)ret; +} \ No newline at end of file diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h index 3e862c3db..fa98ae25e 100644 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h +++ b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h @@ -7,14 +7,6 @@ #ifdef __cplusplus extern "C" { #endif -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: setNumericTableValue - * Signature: (JIID)V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_setNumericTableValue - (JNIEnv *, jobject, jlong, jint, jint, jdouble); - /* * Class: org_apache_spark_ml_util_OneDAL__ * Method: cAddNumericTable @@ -57,12 +49,20 @@ JNIEXPORT jboolean JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cCheckPlat /* * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cNewCSRNumericTable + * Method: cNewCSRNumericTableFloat * Signature: ([F[J[JJJ)J */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTable +JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableFloat (JNIEnv *, jobject, jfloatArray, jlongArray, jlongArray, jlong, jlong); +/* + * Class: org_apache_spark_ml_util_OneDAL__ + * Method: cNewCSRNumericTableDouble + * Signature: ([D[J[JJJ)J + */ +JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableDouble + (JNIEnv *, jobject, jdoubleArray, jlongArray, jlongArray, jlong, jlong); + #ifdef __cplusplus } #endif diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 7bd60e5cc..e681242c1 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -158,12 +158,18 @@ class NaiveBayes @Since("1.5.0") ( $(modelType) match { case Multinomial => - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - val handleWeight = (isDefined(weightCol) && $(weightCol).nonEmpty) - val handleSmoothing = ($(smoothing) != 1.0) - if (isPlatformSupported && !handleWeight && !handleSmoothing) { - trainNaiveBayesDAL(dataset, instr) + val sc = dataset.sparkSession.sparkContext + val isOAPEnabled = sc.conf.getBoolean("spark.oap.mllib.enabled", false) + if (isOAPEnabled) { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + dataset.sparkSession.sparkContext) + val handleWeight = (isDefined(weightCol) && $(weightCol).nonEmpty) + val handleSmoothing = ($(smoothing) != 1.0) + if (isPlatformSupported && !handleWeight && !handleSmoothing) { + trainNaiveBayesDAL(dataset, instr) + } else { + trainDiscreteImpl(dataset, instr) + } } else { trainDiscreteImpl(dataset, instr) } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 0238ef1b1..b9127cb17 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -36,7 +36,7 @@ class NaiveBayesDALImpl(val uid: String, // val featureTables = OneDAL.vectorsToMergedNumericTables(features, executorNum) // val labelTables = OneDAL.doublesToNumericTables(labels, executorNum) - val labeledPointsTables = OneDAL.labeledPointsToMergedNumericTables(labeledPoints, executorNum) + val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables(labeledPoints, executorNum) val results = labeledPointsTables.mapPartitionsWithIndex { case (rank: Int, tables: Iterator[(Long, Long)]) => diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala index f2d0bbe5e..9229d7ff5 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala @@ -34,7 +34,7 @@ class KMeansDALImpl(var nClusters: Int, def train(data: RDD[Vector], instr: Option[Instrumentation]): MLlibKMeansModel = { - val coalescedTables = OneDAL.vectorsToMergedNumericTables(data, executorNum) + val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) val executorIPAddress = Utils.sparkFirstExecutorIP(coalescedTables.sparkContext) val kvsIP = coalescedTables.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala index f2b8645a2..1b6278cbc 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala @@ -36,7 +36,7 @@ class PCADALImpl(val k: Int, val normalizedData = normalizeData(data) - val coalescedTables = OneDAL.vectorsToMergedNumericTables(normalizedData, executorNum) + val coalescedTables = OneDAL.rddVectorToMergedTables(normalizedData, executorNum) val executorIPAddress = Utils.sparkFirstExecutorIP(coalescedTables.sparkContext) val kvsIP = coalescedTables.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala index 1e16c97c3..01bf55811 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala @@ -222,7 +222,7 @@ class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( data: RDD[Rating[ID]], rowOffsets += index + 1 val contextLocal = new DaalContext() - val cTable = OneDAL.cNewCSRNumericTable(values, columnIndices, rowOffsets.toArray, + val cTable = OneDAL.cNewCSRNumericTableFloat(values, columnIndices, rowOffsets.toArray, nVectors, csrRowNum) val table = new CSRNumericTable(contextLocal, cTable) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index b2e06c696..efa14c58b 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -16,15 +16,17 @@ package org.apache.spark.ml.util -import com.intel.daal.data_management.data.{HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} +import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext import org.apache.spark.SparkContext -import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, Matrices, Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} import org.apache.spark.storage.StorageLevel +import java.lang import java.util.logging.{Level, Logger} +import scala.collection.mutable.ArrayBuffer object OneDAL { @@ -70,11 +72,7 @@ object OneDAL { resArray } - // def vectorsToCSRNumericTable(vectors: Iterator[Vector]): CSRNumericTable = { - // vectors.foreach { v => - // - // } - // } + def makeNumericTable(cData: Long): NumericTable = { @@ -96,7 +94,7 @@ object OneDAL { arrayVectors.zipWithIndex.foreach { case (v, rowIndex) => for (colIndex <- 0 until numCols) { - setNumericTableValue(matrix.getCNumericTable, rowIndex, colIndex, v(colIndex)) + cSetDouble(matrix.getCNumericTable, rowIndex, colIndex, v(colIndex)) } } @@ -114,7 +112,24 @@ object OneDAL { } } - def doublesToNumericTables(doubles: RDD[Double], executorNum: Int): RDD[Long] = { + private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrixLabel = new DALMatrix(context, classOf[lang.Double], + 1, points.length, NumericTable.AllocationFlag.DoAllocate) + + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + points.zipWithIndex.foreach { case (point: Double, index: Int) => + cSetDouble(matrixLabel.getCNumericTable, index, 0, point) + } + + matrixLabel + } + + def rddDoubleToNumericTables(doubles: RDD[Double], executorNum: Int): RDD[Long] = { require(executorNum > 0) val doublesTables = doubles.repartition(executorNum).mapPartitions { it: Iterator[Double] => val data = it.toArray @@ -136,8 +151,8 @@ object OneDAL { doublesTables } - def labeledPointsToMergedNumericTables(labeledPoints: RDD[(Vector, Double)], - executorNum: Int): RDD[(Long, Long)] = { + def rddLabeledPointToMergedTables(labeledPoints: RDD[(Vector, Double)], + executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) logger.info(s"Processing partitions with $executorNum executors") @@ -145,36 +160,30 @@ object OneDAL { val tables = labeledPoints.repartition(executorNum).mapPartitions { it: Iterator[(Vector, Double)] => val points: Array[(Vector, Double)] = it.toArray - val numColumns = points(0)._1.size - - // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc - val context = new DaalContext() - val matrixFeature = new DALMatrix(context, classOf[java.lang.Double], - numColumns, points.length, NumericTable.AllocationFlag.DoAllocate) - val matrixLabel = new DALMatrix(context, classOf[java.lang.Double], - 1, points.length, NumericTable.AllocationFlag.DoAllocate) - - // oneDAL libs should be loaded by now, loading other native libs - logger.info("Loading native libraries") - LibLoader.loadLibraries() - - points.zipWithIndex.foreach { case (point: (Vector, Double), index: Int) => - val rowArray = point._1.toArray - cSetDoubleBatch(matrixFeature.getCNumericTable, index, rowArray, 1, numColumns) - cSetDouble(matrixLabel.getCNumericTable, index, 0, point._2) + + val features = points.map(_._1) + val labels = points.map(_._2) + + val numColumns = features(0).size + + val featuresTable = if (features(0).isInstanceOf[DenseVector]) { + vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) + } else { + vectorsToSparseNumericTable(features, numColumns) } - println(matrixFeature.getNumberOfRows, matrixFeature.getNumberOfColumns) - println(matrixLabel.getNumberOfRows, matrixLabel.getNumberOfColumns) + val labelsTable = doubleArrayToNumericTable(labels) - Iterator((matrixFeature.getCNumericTable, matrixLabel.getCNumericTable)) + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) } tables.count() tables } - def vectorsToMergedNumericTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { + + + def rddVectorToMergedTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { require(executorNum > 0) logger.info(s"Processing partitions with $executorNum executors") @@ -206,24 +215,8 @@ object OneDAL { logger.info(s"Partition index: $index, numCols: $numCols, numRows: $numRows") - // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc - val context = new DaalContext() - val matrix = new DALMatrix(context, classOf[java.lang.Double], - numCols.toLong, numRows.toLong, NumericTable.AllocationFlag.DoAllocate) - - // oneDAL libs should be loaded by now, loading other native libs - logger.info("Loading native libraries") - LibLoader.loadLibraries() - - var dalRow = 0 - - it.foreach { curVector => - val rowArray = curVector.toArray - OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) - dalRow += 1 - } - - matrix.getCNumericTable + val table = vectorsToDenseNumericTable(it, numRows, numCols) + table.getCNumericTable }.setName("numericTables").cache() numericTables.count() @@ -250,7 +243,68 @@ object OneDAL { coalescedTables } - @native def setNumericTableValue(numTableAddr: Long, rowIndex: Int, colIndex: Int, value: Double) + private def vectorsToDenseNumericTable(it: Iterator[Vector], numRows: Int, numCols: Int): NumericTable = { + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrix = new DALMatrix(context, classOf[lang.Double], + numCols.toLong, numRows.toLong, NumericTable.AllocationFlag.DoAllocate) + + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + var dalRow = 0 + + it.foreach { curVector => + val rowArray = curVector.toArray + OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) + dalRow += 1 + } + matrix + } + private def vectorsToSparseNumericTable(vectors: Array[Vector], nFeatures: Long): CSRNumericTable = { + require(vectors(0).isInstanceOf[SparseVector], "vectors should be sparse") + + val ratingsNum = vectors.map(_.numActives).sum + val csrRowNum = vectors.length + val values = Array.fill(ratingsNum) { + 0.0 + } + val columnIndices = Array.fill(ratingsNum) { + 0L + } + val rowOffsets = ArrayBuffer[Long](1L) + + var indexValues = 0 + var curRow = 0L + + // Converted to one CSRNumericTable + for (row <- 0 until vectors.length) { + val rowVector = vectors(row) + rowVector.foreachActive { (column, value) => + values(indexValues) = value + // one-based indexValues + columnIndices(indexValues) = column + 1 + + if (row > curRow) { + curRow = row + // one-based indexValues + rowOffsets += indexValues + 1 + } + + indexValues = indexValues + 1 + } + } + // one-based row indexValues + rowOffsets += indexValues + 1 + + val contextLocal = new DaalContext() + val cTable = OneDAL.cNewCSRNumericTableDouble(values, columnIndices, rowOffsets.toArray, + nFeatures, csrRowNum) + val table = new CSRNumericTable(contextLocal, cTable) + + table + } @native def cAddNumericTable(cObject: Long, numericTableAddr: Long) @@ -263,7 +317,10 @@ object OneDAL { @native def cCheckPlatformCompatibility(): Boolean - @native def cNewCSRNumericTable(data: Array[Float], - colIndices: Array[Long], rowOffsets: Array[Long], - nFeatures: Long, nVectors: Long): Long + @native def cNewCSRNumericTableFloat(data: Array[Float], + colIndices: Array[Long], rowOffsets: Array[Long], + nFeatures: Long, nVectors: Long): Long + @native def cNewCSRNumericTableDouble(data: Array[Double], + colIndices: Array[Long], rowOffsets: Array[Long], + nFeatures: Long, nVectors: Long): Long } From de55829dc6bbcd79cf1e05b54261c549b29fc6a3 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 18 May 2021 10:22:12 +0800 Subject: [PATCH 17/29] Add Profiler --- .../src/main/native/NaiveBayesDALImpl.cpp | 65 ++++++++++++++----- mllib-dal/src/main/native/Profile.hpp | 34 ++++++++++ 2 files changed, 84 insertions(+), 15 deletions(-) create mode 100644 mllib-dal/src/main/native/Profile.hpp diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index f7642f1ae..f5a5fe4fa 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -4,6 +4,12 @@ #include "org_apache_spark_ml_classification_NaiveBayesDALImpl.h" #include "service.h" +#define PROFILE 1 + +#ifdef PROFILE +#include "Profile.hpp" +#endif + using namespace std; using namespace daal; using namespace daal::algorithms; @@ -16,6 +22,10 @@ template static training::ResultPtr trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, const NumericTablePtr &labelsTab, int nClasses) { +#ifdef PROFILE + Profiler profiler("NaiveBayes"); +#endif + auto rankId = comm.rank(); auto nBlocks = comm.size(); @@ -28,9 +38,17 @@ trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, localAlgorithm.input.set(classifier::training::data, featuresTab); localAlgorithm.input.set(classifier::training::labels, labelsTab); +#ifdef PROFILE + profiler.startProfile("local step compute"); +#endif + /* Train the Naive Bayes model on local nodes */ localAlgorithm.compute(); +#ifdef PROFILE + profiler.endProfile(); +#endif + /* Serialize partial results required by step 2 */ services::SharedPtr serializedData; InputDataArchive dataArch; @@ -48,13 +66,21 @@ trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, new daal::byte[perNodeArchLength]); dataArch.copyArchiveToArray(nodeResults.get(), perNodeArchLength); - /* Transfer partial results to step 2 on the root node */ - // MPI_Gather(nodeResults.get(), perNodeArchLength, MPI_CHAR, - // serializedData.get(), perNodeArchLength, MPI_CHAR, mpi_root, - // MPI_COMM_WORLD); +/* Transfer partial results to step 2 on the root node */ +// MPI_Gather(nodeResults.get(), perNodeArchLength, MPI_CHAR, +// serializedData.get(), perNodeArchLength, MPI_CHAR, mpi_root, +// MPI_COMM_WORLD); +#ifdef PROFILE + profiler.startProfile("ccl::gather"); +#endif + ccl::gather(nodeResults.get(), perNodeArchLength, serializedData.get(), perNodeArchLength, comm) .wait(); + +#ifdef PROFILE + profiler.endProfile(); +#endif } if (rankId == ccl_root) { @@ -79,10 +105,17 @@ trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, dataForStep2FromStep1); } +#ifdef PROFILE + profiler.startProfile("master step compute"); +#endif /* Merge and finalizeCompute the Naive Bayes model on the master node */ masterAlgorithm.compute(); masterAlgorithm.finalizeCompute(); +#ifdef PROFILE + profiler.endProfile(); +#endif + /* Retrieve the algorithm results */ training::ResultPtr trainingResult = masterAlgorithm.getResult(); @@ -100,8 +133,8 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( JNIEnv *env, jobject obj, jlong pFeaturesTab, jlong pLabelsTab, jint class_num, jint executor_num, jint executor_cores, jobject resultObj) { - - ccl::communicator &comm = getComm(); + + ccl::communicator &comm = getComm(); auto rankId = comm.rank(); NumericTablePtr featuresTab = *((NumericTablePtr *)pFeaturesTab); @@ -119,10 +152,13 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( training::ResultPtr trainingResult; if (featuresTab->getDataLayout() == NumericTable::StorageLayout::csrArray) { cout << "oneDAL (native): training model with fastCSR method" << endl; - trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); + trainingResult = trainModel(comm, featuresTab, + labelsTab, class_num); } else { - cout << "oneDAL (native): training model with defaultDense method" << endl; - trainingResult = trainModel(comm, featuresTab, labelsTab, class_num); + cout << "oneDAL (native): training model with defaultDense method" + << endl; + trainingResult = trainModel( + comm, featuresTab, labelsTab, class_num); } cout << "oneDAL (native): training model finished" << endl; @@ -130,12 +166,13 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( if (rankId == ccl_root) { multinomial_naive_bayes::ModelPtr model = trainingResult->get(classifier::training::model); - + // auto pi = model->getLogP(); // auto theta = model->getLogTheta(); // printNumericTable(pi, "log of class priors", 10, 20); - // printNumericTable(theta, "log of class conditional probabilities", 10, 20); + // printNumericTable(theta, "log of class conditional probabilities", + // 10, 20); // Return all log of class priors (LogP) and log of class conditional // probabilities (LogTheta) @@ -148,10 +185,8 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( jfieldID thetaNumericTableField = env->GetFieldID(clazz, "thetaNumericTable", "J"); - NumericTablePtr *pi = - new NumericTablePtr(model->getLogP()); - NumericTablePtr *theta = - new NumericTablePtr(model->getLogTheta()); + NumericTablePtr *pi = new NumericTablePtr(model->getLogP()); + NumericTablePtr *theta = new NumericTablePtr(model->getLogTheta()); env->SetLongField(resultObj, piNumericTableField, (jlong)pi); env->SetLongField(resultObj, thetaNumericTableField, (jlong)theta); diff --git a/mllib-dal/src/main/native/Profile.hpp b/mllib-dal/src/main/native/Profile.hpp new file mode 100644 index 000000000..1aa01fc97 --- /dev/null +++ b/mllib-dal/src/main/native/Profile.hpp @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + +class Profiler { + public: + Profiler(std::string s) : subject(s) {} + + void startProfile(std::string s = "") { + action = s; + std::cout << subject << " (native): start" << action << std::endl; + startTime = std::chrono::high_resolution_clock::now(); + } + + void endProfile() { + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast( + end_time - startTime) + .count(); + std::cout << subject << " (native): " << action << "took " << duration + << " secs" << std::endl; + } + + void println(std::string msg) { + std::cout << subject << " (native): " << msg << std::endl; + } + + private: + std::string subject; + std::string action; + std::chrono::system_clock::time_point startTime; +}; From 69a9a238f38eba847f8c2c67d2dc6ea88e9cfe2e Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 18 May 2021 17:31:22 +0800 Subject: [PATCH 18/29] Fix profiler duration --- mllib-dal/src/main/native/NaiveBayesDALImpl.cpp | 8 ++++++++ mllib-dal/src/main/native/OneDAL.cpp | 16 ++++++++-------- mllib-dal/src/main/native/Profile.hpp | 6 +++--- .../ml/classification/NaiveBayes.scala | 8 +++++++- .../scala/org/apache/spark/ml/util/OneDAL.scala | 5 ++++- 5 files changed, 30 insertions(+), 13 deletions(-) diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index f5a5fe4fa..baf1ae2e8 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -1,4 +1,5 @@ #include +#include #include "OneCCL.h" #include "org_apache_spark_ml_classification_NaiveBayesDALImpl.h" @@ -148,6 +149,8 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( cout << "oneDAL (native): Number of CPU threads used: " << nThreadsNew << endl; +auto t1 = std::chrono::high_resolution_clock::now(); + // Support both dense and csr numeric table training::ResultPtr trainingResult; if (featuresTab->getDataLayout() == NumericTable::StorageLayout::csrArray) { @@ -163,6 +166,11 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( cout << "oneDAL (native): training model finished" << endl; +auto t2 = std::chrono::high_resolution_clock::now(); + + std::cout << "training took " << (float)std::chrono::duration_cast(t2 - t1).count() / 1000 << " secs" + << std::endl; + if (rankId == ccl_root) { multinomial_naive_bayes::ModelPtr model = trainingResult->get(classifier::training::model); diff --git a/mllib-dal/src/main/native/OneDAL.cpp b/mllib-dal/src/main/native/OneDAL.cpp index 992fcef2f..c684bda1f 100644 --- a/mllib-dal/src/main/native/OneDAL.cpp +++ b/mllib-dal/src/main/native/OneDAL.cpp @@ -32,13 +32,13 @@ extern bool daal_check_is_intel_cpu(); * Method: cSetDouble * Signature: (JIID)V */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDouble - (JNIEnv *env, jobject, jlong numTableAddr, jint row, jint column, jdouble value) { - HomogenNumericTable *nt = - static_cast *>( - ((SerializationIfacePtr *)numTableAddr)->get()); - (*nt)[row][column] = (double)value; - +JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDouble( + JNIEnv *env, jobject, jlong numTableAddr, jint row, jint column, + jdouble value) { + HomogenNumericTable *nt = + static_cast *>( + ((SerializationIfacePtr *)numTableAddr)->get()); + (*nt)[row][column] = (double)value; } /* @@ -163,7 +163,7 @@ Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableDouble( resultData, resultColIndices, resultRowOffsets, nFeatures, nVectors); numericTable->allocateDataMemory(numData); numericTable->getArrays(&resultData, &resultColIndices, - &resultRowOffsets); + &resultRowOffsets); size_t *pRowOffsets = (size_t *)env->GetLongArrayElements(rowOffsets, 0); size_t *pColIndices = (size_t *)env->GetLongArrayElements(colIndices, 0); diff --git a/mllib-dal/src/main/native/Profile.hpp b/mllib-dal/src/main/native/Profile.hpp index 1aa01fc97..85f85dd5f 100644 --- a/mllib-dal/src/main/native/Profile.hpp +++ b/mllib-dal/src/main/native/Profile.hpp @@ -10,16 +10,16 @@ class Profiler { void startProfile(std::string s = "") { action = s; - std::cout << subject << " (native): start" << action << std::endl; + std::cout << subject << " (native): start " << action << std::endl; startTime = std::chrono::high_resolution_clock::now(); } void endProfile() { auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast( + auto duration = std::chrono::duration_cast( end_time - startTime) .count(); - std::cout << subject << " (native): " << action << "took " << duration + std::cout << subject << " (native): " << action << " took " << (float)duration / 1000 << " secs" << std::endl; } diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index e681242c1..9c4c5775d 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -160,7 +160,7 @@ class NaiveBayes @Since("1.5.0") ( case Multinomial => val sc = dataset.sparkSession.sparkContext val isOAPEnabled = sc.conf.getBoolean("spark.oap.mllib.enabled", false) - if (isOAPEnabled) { + val model = if (isOAPEnabled) { val isPlatformSupported = Utils.checkClusterPlatformCompatibility( dataset.sparkSession.sparkContext) val handleWeight = (isDefined(weightCol) && $(weightCol).nonEmpty) @@ -173,6 +173,8 @@ class NaiveBayes @Since("1.5.0") ( } else { trainDiscreteImpl(dataset, instr) } + print(model.pi.toArray.slice(0, 20).mkString(" ")) + model case Bernoulli | Complement => trainDiscreteImpl(dataset, instr) case Gaussian => @@ -192,6 +194,10 @@ class NaiveBayes @Since("1.5.0") ( logInfo(s"NaiveBayesDAL fit using $executor_num Executors") + dataset.cache() + dataset.count() + + // Todo: optimize getting num of classes val numClasses = getNumClasses(dataset) val labeledPoints: RDD[(Vector, Double)] = dataset diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index efa14c58b..6f5911ef3 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -175,7 +175,8 @@ object OneDAL { val labelsTable = doubleArrayToNumericTable(labels) Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) - } + }.cache() + tables.count() tables @@ -265,6 +266,8 @@ object OneDAL { private def vectorsToSparseNumericTable(vectors: Array[Vector], nFeatures: Long): CSRNumericTable = { require(vectors(0).isInstanceOf[SparseVector], "vectors should be sparse") + println(s"Features row x column: ${vectors.length} x ${vectors(0).size}") + val ratingsNum = vectors.map(_.numActives).sum val csrRowNum = vectors.length val values = Array.fill(ratingsNum) { From 34d21599aaa22ee06674016429b5e16ae80bf88c Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 19 May 2021 15:56:18 +0800 Subject: [PATCH 19/29] Improve instrumentation --- .../ml/classification/NaiveBayes.scala | 20 ++- .../ml/classification/NaiveBayesDALImpl.scala | 8 +- .../org/apache/spark/ml/util/OneDAL.scala | 170 +++++++++--------- 3 files changed, 104 insertions(+), 94 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 9c4c5775d..a5f9d40ca 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -173,7 +173,7 @@ class NaiveBayes @Since("1.5.0") ( } else { trainDiscreteImpl(dataset, instr) } - print(model.pi.toArray.slice(0, 20).mkString(" ")) + // println(model.pi.toArray.slice(0, 20).mkString(" ")) model case Bernoulli | Complement => trainDiscreteImpl(dataset, instr) @@ -187,19 +187,30 @@ class NaiveBayes @Since("1.5.0") ( private def trainNaiveBayesDAL(dataset: Dataset[_], instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ + + val sc = spark.sparkContext - val sc = dataset.sparkSession.sparkContext val executor_num = Utils.sparkExecutorNum(sc) val executor_cores = Utils.sparkExecutorCores() logInfo(s"NaiveBayesDAL fit using $executor_num Executors") dataset.cache() - dataset.count() + val numSamples = dataset.count() - // Todo: optimize getting num of classes + // Todo: optimize getting num of classes, DAL only support [0..numClasses) as labels + // Should map original labels using StringIndexer val numClasses = getNumClasses(dataset) +// println(dataset.select($(labelCol)).distinct().collect().mkString(" ")) + val numFeatures = dataset.select($(featuresCol)).as[Tuple1[Vector]].take(1)(0)._1.size + + instr.logNumFeatures(numFeatures) + instr.logNumExamples(numSamples) + instr.logNumClasses(numClasses) + val labeledPoints: RDD[(Vector, Double)] = dataset .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), col(getLabelCol)).rdd.map { case Row(feature: Vector, label: Double) => (feature, label) @@ -210,7 +221,6 @@ class NaiveBayes @Since("1.5.0") ( // Set labels to be compatible with old mllib model val labels = (0 until numClasses).map(_.toDouble).toArray - model.setOldLabels(labels) model diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index b9127cb17..0ce2c09c2 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -17,10 +17,9 @@ package org.apache.spark.ml.classification import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.LabeledPoint -import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.ml.linalg.{Matrices, Vector} +import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL, Service} import org.apache.spark.rdd.RDD class NaiveBayesDALImpl(val uid: String, @@ -33,9 +32,6 @@ class NaiveBayesDALImpl(val uid: String, val kvsIPPort = getOneCCLIPPort(labeledPoints) -// val featureTables = OneDAL.vectorsToMergedNumericTables(features, executorNum) -// val labelTables = OneDAL.doublesToNumericTables(labels, executorNum) - val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables(labeledPoints, executorNum) val results = labeledPointsTables.mapPartitionsWithIndex { diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 6f5911ef3..93475ec7a 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -16,18 +16,21 @@ package org.apache.spark.ml.util -import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} +import java.lang +import java.util.logging.{Level, Logger} + +import scala.collection.mutable.ArrayBuffer + +import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, Matrix => DALMatrix, + NumericTable, RowMergedNumericTable} import com.intel.daal.services.DaalContext + import org.apache.spark.SparkContext import org.apache.spark.ml.linalg.{DenseVector, Matrices, Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} import org.apache.spark.storage.StorageLevel -import java.lang -import java.util.logging.{Level, Logger} -import scala.collection.mutable.ArrayBuffer - object OneDAL { private val logger = Logger.getLogger("util.OneDAL") @@ -73,7 +76,6 @@ object OneDAL { } - def makeNumericTable(cData: Long): NumericTable = { val context = new DaalContext() @@ -112,23 +114,6 @@ object OneDAL { } } - private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { - // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc - val context = new DaalContext() - val matrixLabel = new DALMatrix(context, classOf[lang.Double], - 1, points.length, NumericTable.AllocationFlag.DoAllocate) - - // oneDAL libs should be loaded by now, loading other native libs - logger.info("Loading native libraries") - LibLoader.loadLibraries() - - points.zipWithIndex.foreach { case (point: Double, index: Int) => - cSetDouble(matrixLabel.getCNumericTable, index, 0, point) - } - - matrixLabel - } - def rddDoubleToNumericTables(doubles: RDD[Double], executorNum: Int): RDD[Long] = { require(executorNum > 0) val doublesTables = doubles.repartition(executorNum).mapPartitions { it: Iterator[Double] => @@ -182,69 +167,25 @@ object OneDAL { tables } + private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrixLabel = new DALMatrix(context, classOf[lang.Double], + 1, points.length, NumericTable.AllocationFlag.DoAllocate) + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() - def rddVectorToMergedTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { - require(executorNum > 0) - - logger.info(s"Processing partitions with $executorNum executors") - - // Repartition to executorNum if not enough partitions - val dataForConversion = if (vectors.getNumPartitions < executorNum) { - vectors.repartition(executorNum).setName("Repartitioned for conversion").cache() - } else { - vectors - } - - // Get dimensions for each partition - val partitionDims = Utils.getPartitionDims(dataForConversion) - - // Filter out empty partitions - val nonEmptyPartitions = dataForConversion.mapPartitionsWithIndex { - (index: Int, it: Iterator[Vector]) => Iterator(Tuple3(partitionDims(index)._1, index, it)) - }.filter { entry => { - entry._1 > 0 - } - } - - // Convert to RDD[HomogenNumericTable] - val numericTables = nonEmptyPartitions.map { entry => - val numRows = entry._1 - val index = entry._2 - val it = entry._3 - val numCols = partitionDims(index)._2 - - logger.info(s"Partition index: $index, numCols: $numCols, numRows: $numRows") - - val table = vectorsToDenseNumericTable(it, numRows, numCols) - table.getCNumericTable - }.setName("numericTables").cache() - - numericTables.count() - - // Unpersist instances RDD - if (vectors.getStorageLevel != StorageLevel.NONE) { - vectors.unpersist() + points.zipWithIndex.foreach { case (point: Double, index: Int) => + cSetDouble(matrixLabel.getCNumericTable, index, 0, point) } - // Coalesce partitions belonging to the same executor - val coalescedRdd = numericTables.coalesce(executorNum, - partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) - - val coalescedTables = coalescedRdd.mapPartitions { iter => - val context = new DaalContext() - val mergedData = new RowMergedNumericTable(context) - - iter.foreach { address => - OneDAL.cAddNumericTable(mergedData.getCNumericTable, address) - } - Iterator(mergedData.getCNumericTable) - }.cache() - - coalescedTables + matrixLabel } - private def vectorsToDenseNumericTable(it: Iterator[Vector], numRows: Int, numCols: Int): NumericTable = { + private def vectorsToDenseNumericTable(it: Iterator[Vector], + numRows: Int, numCols: Int): NumericTable = { // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc val context = new DaalContext() val matrix = new DALMatrix(context, classOf[lang.Double], @@ -263,7 +204,9 @@ object OneDAL { } matrix } - private def vectorsToSparseNumericTable(vectors: Array[Vector], nFeatures: Long): CSRNumericTable = { + + private def vectorsToSparseNumericTable(vectors: Array[Vector], + nFeatures: Long): CSRNumericTable = { require(vectors(0).isInstanceOf[SparseVector], "vectors should be sparse") println(s"Features row x column: ${vectors.length} x ${vectors(0).size}") @@ -309,6 +252,66 @@ object OneDAL { table } + def rddVectorToMergedTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { + require(executorNum > 0) + + logger.info(s"Processing partitions with $executorNum executors") + + // Repartition to executorNum if not enough partitions + val dataForConversion = if (vectors.getNumPartitions < executorNum) { + vectors.repartition(executorNum).setName("Repartitioned for conversion").cache() + } else { + vectors + } + + // Get dimensions for each partition + val partitionDims = Utils.getPartitionDims(dataForConversion) + + // Filter out empty partitions + val nonEmptyPartitions = dataForConversion.mapPartitionsWithIndex { + (index: Int, it: Iterator[Vector]) => Iterator(Tuple3(partitionDims(index)._1, index, it)) + }.filter { entry => { + entry._1 > 0 + } + } + + // Convert to RDD[HomogenNumericTable] + val numericTables = nonEmptyPartitions.map { entry => + val numRows = entry._1 + val index = entry._2 + val it = entry._3 + val numCols = partitionDims(index)._2 + + logger.info(s"Partition index: $index, numCols: $numCols, numRows: $numRows") + + val table = vectorsToDenseNumericTable(it, numRows, numCols) + table.getCNumericTable + }.setName("numericTables").cache() + + numericTables.count() + + // Unpersist instances RDD + if (vectors.getStorageLevel != StorageLevel.NONE) { + vectors.unpersist() + } + + // Coalesce partitions belonging to the same executor + val coalescedRdd = numericTables.coalesce(executorNum, + partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) + + val coalescedTables = coalescedRdd.mapPartitions { iter => + val context = new DaalContext() + val mergedData = new RowMergedNumericTable(context) + + iter.foreach { address => + OneDAL.cAddNumericTable(mergedData.getCNumericTable, address) + } + Iterator(mergedData.getCNumericTable) + }.cache() + + coalescedTables + } + @native def cAddNumericTable(cObject: Long, numericTableAddr: Long) @native def cSetDouble(numTableAddr: Long, row: Int, column: Int, value: Double) @@ -323,7 +326,8 @@ object OneDAL { @native def cNewCSRNumericTableFloat(data: Array[Float], colIndices: Array[Long], rowOffsets: Array[Long], nFeatures: Long, nVectors: Long): Long + @native def cNewCSRNumericTableDouble(data: Array[Double], - colIndices: Array[Long], rowOffsets: Array[Long], - nFeatures: Long, nVectors: Long): Long + colIndices: Array[Long], rowOffsets: Array[Long], + nFeatures: Long, nVectors: Long): Long } From a943d30beeec7c2b152326bf6068f9ae5ce54bcb Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 19 May 2021 15:59:58 +0800 Subject: [PATCH 20/29] Add NaiveBayesExample --- examples/naive-bayes/build.sh | 3 + examples/naive-bayes/pom.xml | 94 +++++++++++++++++++ examples/naive-bayes/run.sh | 26 +++++ .../spark/examples/ml/NaiveBayesExample.scala | 66 +++++++++++++ 4 files changed, 189 insertions(+) create mode 100755 examples/naive-bayes/build.sh create mode 100644 examples/naive-bayes/pom.xml create mode 100755 examples/naive-bayes/run.sh create mode 100644 examples/naive-bayes/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala diff --git a/examples/naive-bayes/build.sh b/examples/naive-bayes/build.sh new file mode 100755 index 000000000..da373645b --- /dev/null +++ b/examples/naive-bayes/build.sh @@ -0,0 +1,3 @@ +#!/usr/bin/env bash + +mvn clean package diff --git a/examples/naive-bayes/pom.xml b/examples/naive-bayes/pom.xml new file mode 100644 index 000000000..21b6980dc --- /dev/null +++ b/examples/naive-bayes/pom.xml @@ -0,0 +1,94 @@ + + 4.0.0 + + com.intel.oap + oap-mllib-examples + ${oap.version}-with-spark-${spark.version} + jar + + NaiveBayesExample + https://github.com/oap-project/oap-mllib.git + + + UTF-8 + 1.1.0 + 2.12.10 + 2.12 + 3.0.0 + + + + + + org.scala-lang + scala-library + 2.12.10 + + + + com.github.scopt + scopt_2.12 + 3.7.0 + + + + org.apache.spark + spark-sql_2.12 + ${spark.version} + provided + + + + org.apache.spark + spark-mllib_2.12 + ${spark.version} + provided + + + + + + + + org.scala-tools + maven-scala-plugin + 2.15.2 + + + + compile + testCompile + + + + + ${scala.version} + + -target:jvm-1.8 + + + + + maven-assembly-plugin + 3.0.0 + + false + + jar-with-dependencies + + + + + assembly + package + + single + + + + + + + + diff --git a/examples/naive-bayes/run.sh b/examples/naive-bayes/run.sh new file mode 100755 index 000000000..cc20f62a4 --- /dev/null +++ b/examples/naive-bayes/run.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION-with-spark-3.0.0.jar +APP_CLASS=org.apache.spark.examples.ml.NaiveBayesExample +DATA_FILE=data/sample_libsvm_data.txt + +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --conf "spark.oap.mllib.enabled=true" \ + --conf "spark.serializer=org.apache.spark.serializer.KryoSerializer" \ + --conf "spark.default.parallelism=$SPARK_DEFAULT_PARALLELISM" \ + --conf "spark.sql.shuffle.partitions=$SPARK_DEFAULT_PARALLELISM" \ + --conf "spark.driver.extraClassPath=$SPARK_DRIVER_CLASSPATH" \ + --conf "spark.executor.extraClassPath=$SPARK_EXECUTOR_CLASSPATH" \ + --conf "spark.shuffle.reduceLocality.enabled=false" \ + --conf "spark.network.timeout=1200s" \ + --conf "spark.task.maxFailures=1" \ + --jars $OAP_MLLIB_JAR \ + --class $APP_CLASS \ + $APP_JAR $DATA_FILE $K \ + 2>&1 | tee NaiveBayes-$(date +%m%d_%H_%M_%S).log diff --git a/examples/naive-bayes/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/naive-bayes/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala new file mode 100644 index 000000000..4dc6c82bd --- /dev/null +++ b/examples/naive-bayes/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -0,0 +1,66 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.NaiveBayes +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +// $example off$ +import org.apache.spark.sql.SparkSession + +object NaiveBayesExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("NaiveBayesExample") + .getOrCreate() + + if (args.length != 1) { + println("Require data file path as input parameter") + sys.exit(1) + } + + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + val data = spark.read.format("libsvm").load(args(0)) + + // Split the data into training and test sets (30% held out for testing) + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3), seed = 1234L) + + // Train a NaiveBayes model. + val model = new NaiveBayes() + .fit(trainingData) + + // Select example rows to display. + val predictions = model.transform(testData) + predictions.show() + + // Select (prediction, true label) and compute test error + val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println(s"Test set accuracy = $accuracy") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println From 4f34df1ac8e1be36379b204bb45cdd595ddabe4a Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 20 May 2021 22:07:36 +0800 Subject: [PATCH 21/29] todo: can't use merged table for csr, need to optimize csr data conversion --- .../ml/classification/NaiveBayesDALImpl.scala | 1 + .../org/apache/spark/ml/util/OneDAL.scala | 83 +++++++++++++++++-- 2 files changed, 79 insertions(+), 5 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 0ce2c09c2..d34b03286 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -32,6 +32,7 @@ class NaiveBayesDALImpl(val uid: String, val kvsIPPort = getOneCCLIPPort(labeledPoints) +// val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables_repartition(labeledPoints, executorNum) val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables(labeledPoints, executorNum) val results = labeledPointsTables.mapPartitionsWithIndex { diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 93475ec7a..f5bee5b11 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -136,7 +136,7 @@ object OneDAL { doublesTables } - def rddLabeledPointToMergedTables(labeledPoints: RDD[(Vector, Double)], + def rddLabeledPointToMergedTables_repartition(labeledPoints: RDD[(Vector, Double)], executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) @@ -157,6 +157,8 @@ object OneDAL { vectorsToSparseNumericTable(features, numColumns) } + Service.printNumericTable("featuresTable", featuresTable.asInstanceOf[CSRNumericTable]) + val labelsTable = doubleArrayToNumericTable(labels) Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) @@ -167,6 +169,79 @@ object OneDAL { tables } + def rddLabeledPointToMergedTables(labeledPoints: RDD[(Vector, Double)], + executorNum: Int): RDD[(Long, Long)] = { + require(executorNum > 0) + + logger.info(s"Processing partitions with $executorNum executors") + + // Repartition to executorNum if not enough partitions + val dataForConversion = if (labeledPoints.getNumPartitions < executorNum) { + labeledPoints.repartition(executorNum).setName("Repartitioned for conversion").cache() + } else { + labeledPoints + } + + val tables = dataForConversion.mapPartitions { + it: Iterator[(Vector, Double)] => + val points: Array[(Vector, Double)] = it.toArray + + val features = points.map(_._1) + val labels = points.map(_._2) + + val numColumns = features(0).size + + // Not convert empty partitions + if (labels.length !=0) { +// val featuresTable = if (features(0).isInstanceOf[DenseVector]) { +// vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) +// } else { +// vectorsToSparseNumericTable(features, numColumns) +// } +// Service.printNumericTable("featuresTable", featuresTable.asInstanceOf[CSRNumericTable]) + val featuresTable = vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) + Service.printNumericTable("featuresTable", featuresTable) + + val labelsTable = doubleArrayToNumericTable(labels) + + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + } else { + Iterator() + } + }.cache() + + tables.count() + + // Unpersist labeledPoints RDD + if (labeledPoints.getStorageLevel != StorageLevel.NONE) { + labeledPoints.unpersist() + } + + // Coalesce partitions belonging to the same executor +// val coalescedTables = tables.coalesce(executorNum, +// partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) + + val mergedTables = tables.mapPartitions { iter => + val context = new DaalContext() + val mergedFeatures = new RowMergedNumericTable(context) + val mergedLabels = new RowMergedNumericTable(context) + + iter.foreach { case (featureAddr, labelAddr) => + OneDAL.cAddNumericTable(mergedFeatures.getCNumericTable, featureAddr) + OneDAL.cAddNumericTable(mergedLabels.getCNumericTable, labelAddr) + } + + Service.printNumericTable("mergedFeatures", mergedFeatures, 10, 20) + Service.printNumericTable("mergedLabels", mergedLabels, 10, 20) + + Iterator((mergedFeatures.getCNumericTable, mergedLabels.getCNumericTable)) + }.cache() + + mergedTables.count() + + mergedTables + } + private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc val context = new DaalContext() @@ -202,6 +277,7 @@ object OneDAL { OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) dalRow += 1 } + matrix } @@ -270,10 +346,7 @@ object OneDAL { // Filter out empty partitions val nonEmptyPartitions = dataForConversion.mapPartitionsWithIndex { (index: Int, it: Iterator[Vector]) => Iterator(Tuple3(partitionDims(index)._1, index, it)) - }.filter { entry => { - entry._1 > 0 - } - } + }.filter { _._1 > 0 } // Convert to RDD[HomogenNumericTable] val numericTables = nonEmptyPartitions.map { entry => From 93c1593f67aa64b10978e560e86a9d4877531189 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 27 May 2021 12:21:03 +0800 Subject: [PATCH 22/29] Optimize data conversion with dataset --- .../ml/classification/NaiveBayes.scala | 24 +-- .../ml/classification/NaiveBayesDALImpl.scala | 5 +- .../org/apache/spark/ml/util/OneDAL.scala | 204 +++++++++--------- 3 files changed, 114 insertions(+), 119 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index a5f9d40ca..03b620e58 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -29,7 +29,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.util.VersionUtils @@ -197,27 +197,27 @@ class NaiveBayes @Since("1.5.0") ( logInfo(s"NaiveBayesDAL fit using $executor_num Executors") - dataset.cache() +// dataset.cache() + val labelDF: DataFrame = dataset.select($(labelCol)).cache val numSamples = dataset.count() // Todo: optimize getting num of classes, DAL only support [0..numClasses) as labels // Should map original labels using StringIndexer - val numClasses = getNumClasses(dataset) +// val numClasses = getNumClasses(dataset) + val numClasses = 5 // println(dataset.select($(labelCol)).distinct().collect().mkString(" ")) - val numFeatures = dataset.select($(featuresCol)).as[Tuple1[Vector]].take(1)(0)._1.size - - instr.logNumFeatures(numFeatures) - instr.logNumExamples(numSamples) +// val numFeatures = dataset.select($(featuresCol)).as[Tuple1[Vector]].take(1)(0)._1.size +// +// instr.logNumFeatures(numFeatures) +// instr.logNumExamples(numSamples) instr.logNumClasses(numClasses) - val labeledPoints: RDD[(Vector, Double)] = dataset - .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), col(getLabelCol)).rdd.map { - case Row(feature: Vector, label: Double) => (feature, label) - } + val labeledPointsDS = dataset + .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) val model = new NaiveBayesDALImpl(uid, numClasses, - executor_num, executor_cores).train(labeledPoints, Some(instr)) + executor_num, executor_cores).train(labeledPointsDS, Some(instr)) // Set labels to be compatible with old mllib model val labels = (0 until numClasses).map(_.toDouble).toArray diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index d34b03286..5f9454d5b 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.classification import org.apache.spark.internal.Logging +import org.apache.spark.sql.Dataset import org.apache.spark.ml.linalg.{Matrices, Vector} import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} import org.apache.spark.ml.util.Utils.getOneCCLIPPort @@ -27,10 +28,10 @@ class NaiveBayesDALImpl(val uid: String, val executorNum: Int, val executorCores: Int ) extends Serializable with Logging { - def train(labeledPoints: RDD[(Vector, Double)], + def train(labeledPoints: Dataset[_], instr: Option[Instrumentation]): NaiveBayesModel = { - val kvsIPPort = getOneCCLIPPort(labeledPoints) + val kvsIPPort = getOneCCLIPPort(labeledPoints.rdd) // val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables_repartition(labeledPoints, executorNum) val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables(labeledPoints, executorNum) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index f5bee5b11..406403a6c 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -16,21 +16,19 @@ package org.apache.spark.ml.util -import java.lang -import java.util.logging.{Level, Logger} - -import scala.collection.mutable.ArrayBuffer - -import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, Matrix => DALMatrix, - NumericTable, RowMergedNumericTable} +import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext - import org.apache.spark.SparkContext import org.apache.spark.ml.linalg.{DenseVector, Matrices, Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} +import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.storage.StorageLevel +import java.lang +import java.util.logging.{Level, Logger} +import scala.collection.mutable.ArrayBuffer + object OneDAL { private val logger = Logger.getLogger("util.OneDAL") @@ -137,7 +135,7 @@ object OneDAL { } def rddLabeledPointToMergedTables_repartition(labeledPoints: RDD[(Vector, Double)], - executorNum: Int): RDD[(Long, Long)] = { + executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) logger.info(s"Processing partitions with $executorNum executors") @@ -169,79 +167,6 @@ object OneDAL { tables } - def rddLabeledPointToMergedTables(labeledPoints: RDD[(Vector, Double)], - executorNum: Int): RDD[(Long, Long)] = { - require(executorNum > 0) - - logger.info(s"Processing partitions with $executorNum executors") - - // Repartition to executorNum if not enough partitions - val dataForConversion = if (labeledPoints.getNumPartitions < executorNum) { - labeledPoints.repartition(executorNum).setName("Repartitioned for conversion").cache() - } else { - labeledPoints - } - - val tables = dataForConversion.mapPartitions { - it: Iterator[(Vector, Double)] => - val points: Array[(Vector, Double)] = it.toArray - - val features = points.map(_._1) - val labels = points.map(_._2) - - val numColumns = features(0).size - - // Not convert empty partitions - if (labels.length !=0) { -// val featuresTable = if (features(0).isInstanceOf[DenseVector]) { -// vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) -// } else { -// vectorsToSparseNumericTable(features, numColumns) -// } -// Service.printNumericTable("featuresTable", featuresTable.asInstanceOf[CSRNumericTable]) - val featuresTable = vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) - Service.printNumericTable("featuresTable", featuresTable) - - val labelsTable = doubleArrayToNumericTable(labels) - - Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) - } else { - Iterator() - } - }.cache() - - tables.count() - - // Unpersist labeledPoints RDD - if (labeledPoints.getStorageLevel != StorageLevel.NONE) { - labeledPoints.unpersist() - } - - // Coalesce partitions belonging to the same executor -// val coalescedTables = tables.coalesce(executorNum, -// partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) - - val mergedTables = tables.mapPartitions { iter => - val context = new DaalContext() - val mergedFeatures = new RowMergedNumericTable(context) - val mergedLabels = new RowMergedNumericTable(context) - - iter.foreach { case (featureAddr, labelAddr) => - OneDAL.cAddNumericTable(mergedFeatures.getCNumericTable, featureAddr) - OneDAL.cAddNumericTable(mergedLabels.getCNumericTable, labelAddr) - } - - Service.printNumericTable("mergedFeatures", mergedFeatures, 10, 20) - Service.printNumericTable("mergedLabels", mergedLabels, 10, 20) - - Iterator((mergedFeatures.getCNumericTable, mergedLabels.getCNumericTable)) - }.cache() - - mergedTables.count() - - mergedTables - } - private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc val context = new DaalContext() @@ -259,28 +184,6 @@ object OneDAL { matrixLabel } - private def vectorsToDenseNumericTable(it: Iterator[Vector], - numRows: Int, numCols: Int): NumericTable = { - // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc - val context = new DaalContext() - val matrix = new DALMatrix(context, classOf[lang.Double], - numCols.toLong, numRows.toLong, NumericTable.AllocationFlag.DoAllocate) - - // oneDAL libs should be loaded by now, loading other native libs - logger.info("Loading native libraries") - LibLoader.loadLibraries() - - var dalRow = 0 - - it.foreach { curVector => - val rowArray = curVector.toArray - OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) - dalRow += 1 - } - - matrix - } - private def vectorsToSparseNumericTable(vectors: Array[Vector], nFeatures: Long): CSRNumericTable = { require(vectors(0).isInstanceOf[SparseVector], "vectors should be sparse") @@ -328,6 +231,73 @@ object OneDAL { table } + def rddLabeledPointToMergedTables(labeledPoints: Dataset[_], + executorNum: Int): RDD[(Long, Long)] = { + require(executorNum > 0) + + logger.info(s"Processing partitions with $executorNum executors") + + val spark = SparkSession.active + import spark.implicits._ + + // Repartition to executorNum if not enough partitions + val dataForConversion = if (labeledPoints.rdd.getNumPartitions < executorNum) { + labeledPoints.repartition(executorNum).cache() + } else { + labeledPoints + } + + val tables = dataForConversion.toDF().mapPartitions { it: Iterator[Row] => + val rows = it.toArray + + val features = rows.map { + case Row(label: Double, features: Vector) => features + } + + val labels = rows.map { + case Row(label: Double, features: Vector) => label + } + + val numColumns = features(0).size + + val featuresTable = if (features(0).isInstanceOf[DenseVector]) { + vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) + } else { + vectorsToSparseNumericTable(features, numColumns) + } + + val labelsTable = doubleArrayToNumericTable(labels) + + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + }.cache() + + tables.count() + + // Coalesce partitions belonging to the same executor + val coalescedTables = tables.rdd.coalesce(executorNum, + partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) + + val mergedTables = coalescedTables.mapPartitions { iter => + val context = new DaalContext() + val mergedFeatures = new RowMergedNumericTable(context) + val mergedLabels = new RowMergedNumericTable(context) + + iter.foreach { case (featureAddr, labelAddr) => + OneDAL.cAddNumericTable(mergedFeatures.getCNumericTable, featureAddr) + OneDAL.cAddNumericTable(mergedLabels.getCNumericTable, labelAddr) + } + +// Service.printNumericTable("mergedFeatures", mergedFeatures, 10, 20) +// Service.printNumericTable("mergedLabels", mergedLabels, 10, 20) + + Iterator((mergedFeatures.getCNumericTable, mergedLabels.getCNumericTable)) + }.cache() + + mergedTables.count() + + mergedTables + } + def rddVectorToMergedTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { require(executorNum > 0) @@ -346,7 +316,9 @@ object OneDAL { // Filter out empty partitions val nonEmptyPartitions = dataForConversion.mapPartitionsWithIndex { (index: Int, it: Iterator[Vector]) => Iterator(Tuple3(partitionDims(index)._1, index, it)) - }.filter { _._1 > 0 } + }.filter { + _._1 > 0 + } // Convert to RDD[HomogenNumericTable] val numericTables = nonEmptyPartitions.map { entry => @@ -385,6 +357,28 @@ object OneDAL { coalescedTables } + private def vectorsToDenseNumericTable(it: Iterator[Vector], + numRows: Int, numCols: Int): NumericTable = { + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrix = new DALMatrix(context, classOf[lang.Double], + numCols.toLong, numRows.toLong, NumericTable.AllocationFlag.DoAllocate) + + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + var dalRow = 0 + + it.foreach { curVector => + val rowArray = curVector.toArray + OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) + dalRow += 1 + } + + matrix + } + @native def cAddNumericTable(cObject: Long, numericTableAddr: Long) @native def cSetDouble(numTableAddr: Long, row: Int, column: Int, value: Double) From 51e72e895dc9aea5c2cc27a871d4febd682f1b82 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Mon, 31 May 2021 09:57:37 +0800 Subject: [PATCH 23/29] Add time measurement --- .../ml/classification/NaiveBayes.scala | 8 ++++---- .../ml/classification/NaiveBayesDALImpl.scala | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 03b620e58..1463b98c7 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -198,20 +198,20 @@ class NaiveBayes @Since("1.5.0") ( logInfo(s"NaiveBayesDAL fit using $executor_num Executors") // dataset.cache() - val labelDF: DataFrame = dataset.select($(labelCol)).cache - val numSamples = dataset.count() + // val labelDF: DataFrame = dataset.select($(labelCol)).cache + // val numSamples = dataset.count() // Todo: optimize getting num of classes, DAL only support [0..numClasses) as labels // Should map original labels using StringIndexer // val numClasses = getNumClasses(dataset) - val numClasses = 5 + val numClasses = 100 // println(dataset.select($(labelCol)).distinct().collect().mkString(" ")) // val numFeatures = dataset.select($(featuresCol)).as[Tuple1[Vector]].take(1)(0)._1.size // // instr.logNumFeatures(numFeatures) // instr.logNumExamples(numSamples) - instr.logNumClasses(numClasses) + // instr.logNumClasses(numClasses) val labeledPointsDS = dataset .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 5f9454d5b..c710cb4d6 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -42,18 +42,37 @@ class NaiveBayesDALImpl(val uid: String, OneCCL.init(executorNum, rank, kvsIPPort) + val computeStartTime = System.nanoTime() + val result = new NaiveBayesResult cNaiveBayesDALCompute(featureTabAddr, lableTabAddr, classNum, executorNum, executorCores, result) + + val computeEndTime = System.nanoTime() + val durationCompute = (computeEndTime - computeStartTime).toDouble / 1E9 + + println(s"NaiveBayesDAL compute took ${durationCompute} secs") + val ret = if (OneCCL.isRoot()) { + val convResultStartTime = System.nanoTime() + val pi = OneDAL.numericTableNx1ToVector(OneDAL.makeNumericTable(result.piNumericTable)) val theta = OneDAL.numericTableToMatrix(OneDAL.makeNumericTable(result.thetaNumericTable)) + + val convResultEndTime = System.nanoTime() + + val durationCovResult = (convResultEndTime - convResultStartTime).toDouble / 1E9 + + println(s"NaiveBayesDAL result conversion took ${durationCovResult} secs") + Iterator((pi, theta)) + } else { Iterator.empty } + OneCCL.cleanup() ret }.collect() From eb5bb67ca43afb90e9606e34cc9cc4d8a6b32782 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Mon, 31 May 2021 12:03:40 +0800 Subject: [PATCH 24/29] optimize numericTableToMatrix & numericTableNx1ToVector --- .../org/apache/spark/ml/util/OneDAL.scala | 36 ++++++++++++------- 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 406403a6c..23924bf60 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -19,13 +19,14 @@ package org.apache.spark.ml.util import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext import org.apache.spark.SparkContext -import org.apache.spark.ml.linalg.{DenseVector, Matrices, Matrix, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, Matrices, Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.storage.StorageLevel import java.lang +import java.nio.DoubleBuffer import java.util.logging.{Level, Logger} import scala.collection.mutable.ArrayBuffer @@ -37,22 +38,33 @@ object OneDAL { def numericTableToMatrix(table: NumericTable): Matrix = { val numRows = table.getNumberOfRows.toInt val numCols = table.getNumberOfColumns.toInt - val matrix = Matrices.zeros(numRows, numCols) - for (row <- 0 until numRows) { - for (col <- 0 until numCols) { - matrix.update(row, col, table.getDoubleValue(col, row)) - } - } + + var dataDouble: DoubleBuffer = null + // returned DoubleBuffer is ByteByffer, need to copy as double array + dataDouble = table.getBlockOfRows(0, numRows, dataDouble) + val arrayDouble = new Array[Double](numRows * numCols) + dataDouble.get(arrayDouble) + + // Transpose as DAL numeric table is row-major and DenseMatrix is column major + val matrix = new DenseMatrix(numRows, numCols, arrayDouble, isTransposed = true) + + table.releaseBlockOfRows(0, numRows, dataDouble) + matrix } def numericTableNx1ToVector(table: NumericTable): Vector = { val numRows = table.getNumberOfRows.toInt - val internArray = new Array[Double](numRows.toInt) - for (row <- 0 until numRows) { - internArray(row) = table.getDoubleValue(0, row) - } - Vectors.dense(internArray) + + var dataDouble: DoubleBuffer = null + // returned DoubleBuffer is ByteByffer, need to copy as double array + dataDouble = table.getBlockOfColumnValues(0, 0, numRows, dataDouble) + val arrayDouble = new Array[Double](numRows.toInt) + dataDouble.get(arrayDouble) + + table.releaseBlockOfColumnValues(0, 0, numRows, dataDouble) + + Vectors.dense(arrayDouble) } // Convert DAL numeric table to array of vectors From b36e070eaf1ab8122af0fb8bfc6e6614c4ad4052 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Mon, 31 May 2021 15:47:56 +0800 Subject: [PATCH 25/29] use fixed IP PORT --- mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala index 775728aee..d59493f95 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala @@ -28,7 +28,8 @@ object Utils { val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) - val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) + val kvsPortDetected = 5000 +// val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) val kvsPort = data.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) From af417c205d64552fff564c3dffa7d5fa7cdd8f73 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 3 Jun 2021 21:10:38 +0800 Subject: [PATCH 26/29] Add spark.oap.mllib.classification.classes & fix empty partition --- .../ml/classification/NaiveBayes.scala | 7 ++++++- .../org/apache/spark/ml/util/OneDAL.scala | 20 +++++++++++-------- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 1463b98c7..573005696 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -204,7 +204,12 @@ class NaiveBayes @Since("1.5.0") ( // Todo: optimize getting num of classes, DAL only support [0..numClasses) as labels // Should map original labels using StringIndexer // val numClasses = getNumClasses(dataset) - val numClasses = 100 + val confClasses = sc.conf.getInt("spark.oap.mllib.classification.classes", -1) + + val numClasses = confClasses match { + case -1 => getNumClasses(dataset) + case _ => confClasses + } // println(dataset.select($(labelCol)).distinct().collect().mkString(" ")) // val numFeatures = dataset.select($(featuresCol)).as[Tuple1[Vector]].take(1)(0)._1.size diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 23924bf60..d51d0fa83 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -270,17 +270,21 @@ object OneDAL { case Row(label: Double, features: Vector) => label } - val numColumns = features(0).size - - val featuresTable = if (features(0).isInstanceOf[DenseVector]) { - vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) + if (features.size == 0 ) { + Iterator() } else { - vectorsToSparseNumericTable(features, numColumns) - } + val numColumns = features(0).size + + val featuresTable = if (features(0).isInstanceOf[DenseVector]) { + vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) + } else { + vectorsToSparseNumericTable(features, numColumns) + } - val labelsTable = doubleArrayToNumericTable(labels) + val labelsTable = doubleArrayToNumericTable(labels) - Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + } }.cache() tables.count() From 8973f581594eff43b0ad68e32d08520b4887c53a Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 8 Jun 2021 22:15:19 +0800 Subject: [PATCH 27/29] Add rddLabeledPointToSparseTables & isDenseDataset --- .../ml/classification/NaiveBayes.scala | 10 +- .../ml/classification/NaiveBayesDALImpl.scala | 7 +- .../org/apache/spark/ml/util/OneDAL.scala | 115 +++++++++++++----- 3 files changed, 94 insertions(+), 38 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index 573005696..dad3d7c75 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -201,11 +201,13 @@ class NaiveBayes @Since("1.5.0") ( // val labelDF: DataFrame = dataset.select($(labelCol)).cache // val numSamples = dataset.count() - // Todo: optimize getting num of classes, DAL only support [0..numClasses) as labels - // Should map original labels using StringIndexer -// val numClasses = getNumClasses(dataset) + // Todo: optimize getting num of classes + // DAL only support [0..numClasses) as labels, should map original labels using StringIndexer + // A temp spark config to specify numClasses, may be removed in the future val confClasses = sc.conf.getInt("spark.oap.mllib.classification.classes", -1) + // numClasses should be explicitly included in the parquet metadata + // This can be done by applying StringIndexer to the label column val numClasses = confClasses match { case -1 => getNumClasses(dataset) case _ => confClasses @@ -216,7 +218,7 @@ class NaiveBayes @Since("1.5.0") ( // // instr.logNumFeatures(numFeatures) // instr.logNumExamples(numSamples) - // instr.logNumClasses(numClasses) + instr.logNumClasses(numClasses) val labeledPointsDS = dataset .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index c710cb4d6..1f5b05339 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -33,8 +33,11 @@ class NaiveBayesDALImpl(val uid: String, val kvsIPPort = getOneCCLIPPort(labeledPoints.rdd) -// val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables_repartition(labeledPoints, executorNum) - val labeledPointsTables = OneDAL.rddLabeledPointToMergedTables(labeledPoints, executorNum) + val labeledPointsTables = if (OneDAL.isDenseDataset(labeledPoints)) { + OneDAL.rddLabeledPointToMergedTables(labeledPoints, executorNum) + } else { + OneDAL.rddLabeledPointToSparseTables(labeledPoints, executorNum) + } val results = labeledPointsTables.mapPartitionsWithIndex { case (rank: Int, tables: Iterator[(Long, Long)]) => diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index d51d0fa83..58c465607 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -53,6 +53,12 @@ object OneDAL { matrix } + def isDenseDataset(ds: Dataset[_]): Boolean = { + val row = ds.select("features").head() + + row.get(0).isInstanceOf[DenseVector] + } + def numericTableNx1ToVector(table: NumericTable): Vector = { val numRows = table.getNumberOfRows.toInt @@ -146,38 +152,38 @@ object OneDAL { doublesTables } - def rddLabeledPointToMergedTables_repartition(labeledPoints: RDD[(Vector, Double)], - executorNum: Int): RDD[(Long, Long)] = { - require(executorNum > 0) - - logger.info(s"Processing partitions with $executorNum executors") - - val tables = labeledPoints.repartition(executorNum).mapPartitions { - it: Iterator[(Vector, Double)] => - val points: Array[(Vector, Double)] = it.toArray - - val features = points.map(_._1) - val labels = points.map(_._2) - - val numColumns = features(0).size - - val featuresTable = if (features(0).isInstanceOf[DenseVector]) { - vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) - } else { - vectorsToSparseNumericTable(features, numColumns) - } - - Service.printNumericTable("featuresTable", featuresTable.asInstanceOf[CSRNumericTable]) - - val labelsTable = doubleArrayToNumericTable(labels) - - Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) - }.cache() - - tables.count() - - tables - } +// def rddLabeledPointToMergedTables_repartition(labeledPoints: RDD[(Vector, Double)], +// executorNum: Int): RDD[(Long, Long)] = { +// require(executorNum > 0) +// +// logger.info(s"Processing partitions with $executorNum executors") +// +// val tables = labeledPoints.repartition(executorNum).mapPartitions { +// it: Iterator[(Vector, Double)] => +// val points: Array[(Vector, Double)] = it.toArray +// +// val features = points.map(_._1) +// val labels = points.map(_._2) +// +// val numColumns = features(0).size +// +// val featuresTable = if (features(0).isInstanceOf[DenseVector]) { +// vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) +// } else { +// vectorsToSparseNumericTable(features, numColumns) +// } +// +// Service.printNumericTable("featuresTable", featuresTable.asInstanceOf[CSRNumericTable]) +// +// val labelsTable = doubleArrayToNumericTable(labels) +// +// Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) +// }.cache() +// +// tables.count() +// +// tables +// } private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc @@ -243,6 +249,51 @@ object OneDAL { table } + def rddLabeledPointToSparseTables(labeledPoints: Dataset[_], + executorNum: Int): RDD[(Long, Long)] = { + require(executorNum > 0) + + logger.info(s"Processing partitions with $executorNum executors") + + val spark = SparkSession.active + import spark.implicits._ + + labeledPoints.cache().count() + + val labeledPointsRDD = labeledPoints.toDF().map { + case Row(label: Double, features: Vector) => (features, label) + }.rdd + + val tables = labeledPointsRDD + .coalesce(executorNum, partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) + .mapPartitions { it: Iterator[(Vector, Double)] => + val points: Array[(Vector, Double)] = it.toArray + + val features = points.map(_._1) + val labels = points.map(_._2) + + if (features.size == 0 ) { + Iterator() + } else { + val numColumns = features(0).size + +// val featuresTable = if (features(0).isInstanceOf[DenseVector]) { +// vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) +// } else { + val featuresTable = vectorsToSparseNumericTable(features, numColumns) +// } + + val labelsTable = doubleArrayToNumericTable(labels) + + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + } + }.cache() + + tables.count() + + tables + } + def rddLabeledPointToMergedTables(labeledPoints: Dataset[_], executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) From 8056989087ad5f7e252816e28ee9c8ca8f38500a Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 9 Jun 2021 21:08:27 +0800 Subject: [PATCH 28/29] Add rddLabeledPointToSparseTables_shuffle --- .../org/apache/spark/ml/util/OneDAL.scala | 76 +++++++++---------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 58c465607..0ca4819b7 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -152,39 +152,6 @@ object OneDAL { doublesTables } -// def rddLabeledPointToMergedTables_repartition(labeledPoints: RDD[(Vector, Double)], -// executorNum: Int): RDD[(Long, Long)] = { -// require(executorNum > 0) -// -// logger.info(s"Processing partitions with $executorNum executors") -// -// val tables = labeledPoints.repartition(executorNum).mapPartitions { -// it: Iterator[(Vector, Double)] => -// val points: Array[(Vector, Double)] = it.toArray -// -// val features = points.map(_._1) -// val labels = points.map(_._2) -// -// val numColumns = features(0).size -// -// val featuresTable = if (features(0).isInstanceOf[DenseVector]) { -// vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) -// } else { -// vectorsToSparseNumericTable(features, numColumns) -// } -// -// Service.printNumericTable("featuresTable", featuresTable.asInstanceOf[CSRNumericTable]) -// -// val labelsTable = doubleArrayToNumericTable(labels) -// -// Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) -// }.cache() -// -// tables.count() -// -// tables -// } - private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc val context = new DaalContext() @@ -276,13 +243,46 @@ object OneDAL { Iterator() } else { val numColumns = features(0).size + val featuresTable = vectorsToSparseNumericTable(features, numColumns) + val labelsTable = doubleArrayToNumericTable(labels) + + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + } + }.cache() + + tables.count() + + tables + } + + def rddLabeledPointToSparseTables_shuffle(labeledPoints: Dataset[_], + executorNum: Int): RDD[(Long, Long)] = { + require(executorNum > 0) -// val featuresTable = if (features(0).isInstanceOf[DenseVector]) { -// vectorsToDenseNumericTable(features.toIterator, features.length, numColumns) -// } else { - val featuresTable = vectorsToSparseNumericTable(features, numColumns) -// } + logger.info(s"Processing partitions with $executorNum executors") + + val spark = SparkSession.active + import spark.implicits._ + + val labeledPointsRDD = labeledPoints.rdd.map { + case Row(label: Double, features: Vector) => (features, label) + } + + // Repartition to executorNum + val dataForConversion = labeledPointsRDD.repartition(executorNum) + .setName("Repartitioned for conversion") + + val tables = dataForConversion.mapPartitions { it: Iterator[(Vector, Double)] => + val points: Array[(Vector, Double)] = it.toArray + val features = points.map(_._1) + val labels = points.map(_._2) + + if (features.size == 0 ) { + Iterator() + } else { + val numColumns = features(0).size + val featuresTable = vectorsToSparseNumericTable(features, numColumns) val labelsTable = doubleArrayToNumericTable(labels) Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) From 304a7ee06783cff3c85f8156df3319835c52b646 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 9 Jun 2021 21:30:29 +0800 Subject: [PATCH 29/29] code cleanup --- .../src/main/native/NaiveBayesDALImpl.cpp | 24 +-- .../ml/classification/NaiveBayes.scala | 13 +- .../ml/classification/NaiveBayesDALImpl.scala | 16 +- .../org/apache/spark/ml/util/OneDAL.scala | 157 +++++++++--------- 4 files changed, 96 insertions(+), 114 deletions(-) diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index baf1ae2e8..402947444 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -67,14 +67,11 @@ trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, new daal::byte[perNodeArchLength]); dataArch.copyArchiveToArray(nodeResults.get(), perNodeArchLength); -/* Transfer partial results to step 2 on the root node */ -// MPI_Gather(nodeResults.get(), perNodeArchLength, MPI_CHAR, -// serializedData.get(), perNodeArchLength, MPI_CHAR, mpi_root, -// MPI_COMM_WORLD); #ifdef PROFILE profiler.startProfile("ccl::gather"); #endif + /* Transfer partial results to step 2 on the root node */ ccl::gather(nodeResults.get(), perNodeArchLength, serializedData.get(), perNodeArchLength, comm) .wait(); @@ -149,7 +146,7 @@ Java_org_apache_spark_ml_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( cout << "oneDAL (native): Number of CPU threads used: " << nThreadsNew << endl; -auto t1 = std::chrono::high_resolution_clock::now(); + auto t1 = std::chrono::high_resolution_clock::now(); // Support both dense and csr numeric table training::ResultPtr trainingResult; @@ -166,22 +163,19 @@ auto t1 = std::chrono::high_resolution_clock::now(); cout << "oneDAL (native): training model finished" << endl; -auto t2 = std::chrono::high_resolution_clock::now(); + auto t2 = std::chrono::high_resolution_clock::now(); - std::cout << "training took " << (float)std::chrono::duration_cast(t2 - t1).count() / 1000 << " secs" - << std::endl; + std::cout << "training took " + << (float)std::chrono::duration_cast( + t2 - t1) + .count() / + 1000 + << " secs" << std::endl; if (rankId == ccl_root) { multinomial_naive_bayes::ModelPtr model = trainingResult->get(classifier::training::model); - // auto pi = model->getLogP(); - // auto theta = model->getLogTheta(); - - // printNumericTable(pi, "log of class priors", 10, 20); - // printNumericTable(theta, "log of class conditional probabilities", - // 10, 20); - // Return all log of class priors (LogP) and log of class conditional // probabilities (LogTheta) diff --git a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala index dad3d7c75..0e58b9c71 100644 --- a/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala +++ b/mllib-dal/src/main/scala/org/apache/spark-3.0.0/ml/classification/NaiveBayes.scala @@ -197,12 +197,8 @@ class NaiveBayes @Since("1.5.0") ( logInfo(s"NaiveBayesDAL fit using $executor_num Executors") -// dataset.cache() - // val labelDF: DataFrame = dataset.select($(labelCol)).cache - // val numSamples = dataset.count() - - // Todo: optimize getting num of classes // DAL only support [0..numClasses) as labels, should map original labels using StringIndexer + // Todo: optimize getting num of classes // A temp spark config to specify numClasses, may be removed in the future val confClasses = sc.conf.getInt("spark.oap.mllib.classification.classes", -1) @@ -213,12 +209,7 @@ class NaiveBayes @Since("1.5.0") ( case _ => confClasses } -// println(dataset.select($(labelCol)).distinct().collect().mkString(" ")) -// val numFeatures = dataset.select($(featuresCol)).as[Tuple1[Vector]].take(1)(0)._1.size -// -// instr.logNumFeatures(numFeatures) -// instr.logNumExamples(numSamples) - instr.logNumClasses(numClasses) + instr.logNumClasses(numClasses) val labeledPointsDS = dataset .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala index 1f5b05339..b886ee202 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala @@ -17,17 +17,16 @@ package org.apache.spark.ml.classification import org.apache.spark.internal.Logging -import org.apache.spark.sql.Dataset -import org.apache.spark.ml.linalg.{Matrices, Vector} +import org.apache.spark.ml.linalg.Matrices import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Dataset class NaiveBayesDALImpl(val uid: String, val classNum: Int, val executorNum: Int, val executorCores: Int - ) extends Serializable with Logging { + ) extends Serializable with Logging { def train(labeledPoints: Dataset[_], instr: Option[Instrumentation]): NaiveBayesModel = { @@ -50,13 +49,13 @@ class NaiveBayesDALImpl(val uid: String, val result = new NaiveBayesResult cNaiveBayesDALCompute(featureTabAddr, lableTabAddr, classNum, executorNum, executorCores, result) - + val computeEndTime = System.nanoTime() val durationCompute = (computeEndTime - computeStartTime).toDouble / 1E9 - + println(s"NaiveBayesDAL compute took ${durationCompute} secs") - + val ret = if (OneCCL.isRoot()) { val convResultStartTime = System.nanoTime() @@ -70,12 +69,11 @@ class NaiveBayesDALImpl(val uid: String, println(s"NaiveBayesDAL result conversion took ${durationCovResult} secs") Iterator((pi, theta)) - + } else { Iterator.empty } - OneCCL.cleanup() ret }.collect() diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 0ca4819b7..42b1eef35 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.util import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext import org.apache.spark.SparkContext -import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, Matrices, Matrix, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} import org.apache.spark.sql.{Dataset, Row, SparkSession} @@ -152,6 +152,45 @@ object OneDAL { doublesTables } + def rddLabeledPointToSparseTables(labeledPoints: Dataset[_], + executorNum: Int): RDD[(Long, Long)] = { + require(executorNum > 0) + + logger.info(s"Processing partitions with $executorNum executors") + + val spark = SparkSession.active + import spark.implicits._ + + labeledPoints.cache().count() + + val labeledPointsRDD = labeledPoints.toDF().map { + case Row(label: Double, features: Vector) => (features, label) + }.rdd + + val tables = labeledPointsRDD + .coalesce(executorNum, partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) + .mapPartitions { it: Iterator[(Vector, Double)] => + val points: Array[(Vector, Double)] = it.toArray + + val features = points.map(_._1) + val labels = points.map(_._2) + + if (features.size == 0) { + Iterator() + } else { + val numColumns = features(0).size + val featuresTable = vectorsToSparseNumericTable(features, numColumns) + val labelsTable = doubleArrayToNumericTable(labels) + + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + } + }.cache() + + tables.count() + + tables + } + private def doubleArrayToNumericTable(points: Array[Double]): NumericTable = { // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc val context = new DaalContext() @@ -216,54 +255,14 @@ object OneDAL { table } - def rddLabeledPointToSparseTables(labeledPoints: Dataset[_], - executorNum: Int): RDD[(Long, Long)] = { - require(executorNum > 0) - - logger.info(s"Processing partitions with $executorNum executors") - - val spark = SparkSession.active - import spark.implicits._ - - labeledPoints.cache().count() - - val labeledPointsRDD = labeledPoints.toDF().map { - case Row(label: Double, features: Vector) => (features, label) - }.rdd - - val tables = labeledPointsRDD - .coalesce(executorNum, partitionCoalescer = Some(new ExecutorInProcessCoalescePartitioner())) - .mapPartitions { it: Iterator[(Vector, Double)] => - val points: Array[(Vector, Double)] = it.toArray - - val features = points.map(_._1) - val labels = points.map(_._2) - - if (features.size == 0 ) { - Iterator() - } else { - val numColumns = features(0).size - val featuresTable = vectorsToSparseNumericTable(features, numColumns) - val labelsTable = doubleArrayToNumericTable(labels) - - Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) - } - }.cache() - - tables.count() - - tables - } - def rddLabeledPointToSparseTables_shuffle(labeledPoints: Dataset[_], - executorNum: Int): RDD[(Long, Long)] = { + executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) logger.info(s"Processing partitions with $executorNum executors") val spark = SparkSession.active - import spark.implicits._ - + val labeledPointsRDD = labeledPoints.rdd.map { case Row(label: Double, features: Vector) => (features, label) } @@ -271,21 +270,21 @@ object OneDAL { // Repartition to executorNum val dataForConversion = labeledPointsRDD.repartition(executorNum) .setName("Repartitioned for conversion") - + val tables = dataForConversion.mapPartitions { it: Iterator[(Vector, Double)] => - val points: Array[(Vector, Double)] = it.toArray + val points: Array[(Vector, Double)] = it.toArray - val features = points.map(_._1) - val labels = points.map(_._2) + val features = points.map(_._1) + val labels = points.map(_._2) - if (features.size == 0 ) { - Iterator() - } else { - val numColumns = features(0).size - val featuresTable = vectorsToSparseNumericTable(features, numColumns) - val labelsTable = doubleArrayToNumericTable(labels) + if (features.size == 0) { + Iterator() + } else { + val numColumns = features(0).size + val featuresTable = vectorsToSparseNumericTable(features, numColumns) + val labelsTable = doubleArrayToNumericTable(labels) - Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) + Iterator((featuresTable.getCNumericTable, labelsTable.getCNumericTable)) } }.cache() @@ -321,7 +320,7 @@ object OneDAL { case Row(label: Double, features: Vector) => label } - if (features.size == 0 ) { + if (features.size == 0) { Iterator() } else { val numColumns = features(0).size @@ -354,8 +353,8 @@ object OneDAL { OneDAL.cAddNumericTable(mergedLabels.getCNumericTable, labelAddr) } -// Service.printNumericTable("mergedFeatures", mergedFeatures, 10, 20) -// Service.printNumericTable("mergedLabels", mergedLabels, 10, 20) + // Service.printNumericTable("mergedFeatures", mergedFeatures, 10, 20) + // Service.printNumericTable("mergedLabels", mergedLabels, 10, 20) Iterator((mergedFeatures.getCNumericTable, mergedLabels.getCNumericTable)) }.cache() @@ -365,6 +364,28 @@ object OneDAL { mergedTables } + private def vectorsToDenseNumericTable(it: Iterator[Vector], + numRows: Int, numCols: Int): NumericTable = { + // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc + val context = new DaalContext() + val matrix = new DALMatrix(context, classOf[lang.Double], + numCols.toLong, numRows.toLong, NumericTable.AllocationFlag.DoAllocate) + + // oneDAL libs should be loaded by now, loading other native libs + logger.info("Loading native libraries") + LibLoader.loadLibraries() + + var dalRow = 0 + + it.foreach { curVector => + val rowArray = curVector.toArray + OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) + dalRow += 1 + } + + matrix + } + def rddVectorToMergedTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { require(executorNum > 0) @@ -424,28 +445,6 @@ object OneDAL { coalescedTables } - private def vectorsToDenseNumericTable(it: Iterator[Vector], - numRows: Int, numCols: Int): NumericTable = { - // Build DALMatrix, this will load libJavaAPI, libtbb, libtbbmalloc - val context = new DaalContext() - val matrix = new DALMatrix(context, classOf[lang.Double], - numCols.toLong, numRows.toLong, NumericTable.AllocationFlag.DoAllocate) - - // oneDAL libs should be loaded by now, loading other native libs - logger.info("Loading native libraries") - LibLoader.loadLibraries() - - var dalRow = 0 - - it.foreach { curVector => - val rowArray = curVector.toArray - OneDAL.cSetDoubleBatch(matrix.getCNumericTable, dalRow, rowArray, 1, numCols) - dalRow += 1 - } - - matrix - } - @native def cAddNumericTable(cObject: Long, numericTableAddr: Long) @native def cSetDouble(numTableAddr: Long, row: Int, column: Int, value: Double)