From 02be0335ed298911c22eda95c091b17d57ead62c Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Wed, 3 Nov 2021 16:54:38 +0800 Subject: [PATCH 01/13] 1. enable Summarizer with one API 2. add Summarizer example Signed-off-by: minmingzhu --- examples/summarizer/IntelGpuResourceFile.json | 1 + examples/summarizer/build.sh | 3 + examples/summarizer/pom.xml | 94 ++++++ examples/summarizer/run-gpu-standalone.sh | 41 +++ examples/summarizer/run.sh | 30 ++ .../ml/SummaryStatisticsExample.scala | 67 +++++ .../spark/mllib/stat/SummarizerResult.java | 9 + mllib-dal/src/main/native/Makefile | 6 +- .../src/main/native/SummarizerDALImpl.cpp | 250 ++++++++++++++++ mllib-dal/src/main/native/build-jni.sh | 3 +- ...pache_spark_mllib_stat_SummarizerDALImpl.h | 21 ++ .../org/apache/spark/ml/util/Utils.scala | 4 + .../spark/mllib/stat/SummarizerDALImpl.scala | 118 ++++++++ .../MultivariateStatisticalDALSummary.scala | 65 +++++ .../apache/spark/mllib/stat/Statistics.scala | 275 ++++++++++++++++++ .../MultivariateStatisticalDALSummary.scala | 65 +++++ .../apache/spark/mllib/stat/Statistics.scala | 275 ++++++++++++++++++ .../MultivariateStatisticalDALSummary.scala | 65 +++++ .../apache/spark/mllib/stat/Statistics.scala | 275 ++++++++++++++++++ .../MultivariateStatisticalDALSummary.scala | 65 +++++ .../apache/spark/mllib/stat/Statistics.scala | 275 ++++++++++++++++++ 21 files changed, 2004 insertions(+), 3 deletions(-) create mode 100644 examples/summarizer/IntelGpuResourceFile.json create mode 100644 examples/summarizer/build.sh create mode 100644 examples/summarizer/pom.xml create mode 100755 examples/summarizer/run-gpu-standalone.sh create mode 100644 examples/summarizer/run.sh create mode 100644 examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala create mode 100644 mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java create mode 100644 mllib-dal/src/main/native/SummarizerDALImpl.cpp create mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala create mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala create mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala create mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala create mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala diff --git a/examples/summarizer/IntelGpuResourceFile.json b/examples/summarizer/IntelGpuResourceFile.json new file mode 100644 index 000000000..4b5c3cc98 --- /dev/null +++ b/examples/summarizer/IntelGpuResourceFile.json @@ -0,0 +1 @@ +[{"id":{"componentName": "spark.worker","resourceName":"gpu"},"addresses":["0","1","2","3"]}] diff --git a/examples/summarizer/build.sh b/examples/summarizer/build.sh new file mode 100644 index 000000000..da373645b --- /dev/null +++ b/examples/summarizer/build.sh @@ -0,0 +1,3 @@ +#!/usr/bin/env bash + +mvn clean package diff --git a/examples/summarizer/pom.xml b/examples/summarizer/pom.xml new file mode 100644 index 000000000..2793bc681 --- /dev/null +++ b/examples/summarizer/pom.xml @@ -0,0 +1,94 @@ + + 4.0.0 + + com.intel.oap + oap-mllib-examples + 1.2.0 + jar + + SummaryStatisticsExample + https://github.com/oap-project/oap-mllib.git + + + UTF-8 + 1.2.0 + 2.12.10 + 2.12 + 3.1.1 + + + + + + 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} + + + + + org.apache.spark + spark-mllib_2.12 + ${spark.version} + + + + + + + + + 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/summarizer/run-gpu-standalone.sh b/examples/summarizer/run-gpu-standalone.sh new file mode 100755 index 000000000..853b479db --- /dev/null +++ b/examples/summarizer/run-gpu-standalone.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + +# Data file is from Spark Examples (data/mllib/sample_kmeans_data.txt) and put in examples/data +# The data file should be copied to $HDFS_ROOT before running examples + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar +APP_CLASS=org.apache.spark.examples.ml.SummaryStatisticsExample + +USE_GPU=true +RESOURCE_FILE=$PWD/IntelGpuResourceFile.json +WORKER_GPU_AMOUNT=4 +EXECUTOR_GPU_AMOUNT=1 +TASK_GPU_AMOUNT=1 + +# Should run in standalone mode +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --total-executor-cores $SPARK_TOTAL_CORES \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --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.oap.mllib.useGPU=$USE_GPU" \ + --conf "spark.worker.resourcesFile=$RESOURCE_FILE" \ + --conf "spark.worker.resource.gpu.amount=$WORKER_GPU_AMOUNT" \ + --conf "spark.executor.resource.gpu.amount=$EXECUTOR_GPU_AMOUNT" \ + --conf "spark.task.resource.gpu.amount=$TASK_GPU_AMOUNT" \ + --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 \ + 2>&1 | tee Summarizer-$(date +%m%d_%H_%M_%S).log + diff --git a/examples/summarizer/run.sh b/examples/summarizer/run.sh new file mode 100644 index 000000000..6a8bab7d3 --- /dev/null +++ b/examples/summarizer/run.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar +APP_CLASS=org.apache.spark.examples.ml.SummaryStatisticsExample + +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --total-executor-cores $SPARK_TOTAL_CORES \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --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" \ + --conf "spark.eventLog.enabled=true" \ + --conf "spark.eventLog.dir=hdfs://beaver:9000/spark-history-server" \ + --conf "spark.history.fs.logDirectory=hdfs://beaver:9000/spark-history-server" \ + --conf "spark.driver.log.dfsDir=/user/spark/driverLogs" \ + --jars $OAP_MLLIB_JAR \ + --class $APP_CLASS \ + $APP_JAR \ + 2>&1 | tee Summarizer-$(date +%m%d_%H_%M_%S).log \ No newline at end of file diff --git a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala new file mode 100644 index 000000000..a173e2782 --- /dev/null +++ b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala @@ -0,0 +1,67 @@ +/* + * 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 + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector => NewVector} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.SparkSession +import org.apache.spark.rdd.RDD + +// $example off$ + +object SummaryStatisticsExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("SummaryStatisticsExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = sc.parallelize( Seq( + Vectors.dense(5.308206,9.869278,1.018934,4.292158,6.081011,6.585723,2.411094,4.767308,-3.256320,-6.029562), + Vectors.dense(7.279464,0.390664,-9.619284,3.435376,-4.769490,-4.873188,-0.118791,-5.117316,-0.418655,-0.475422), + Vectors.dense(-6.615791,-6.191542,0.402459,-9.743521,-9.990568,9.105346,1.691312,-2.605659,9.534952,-7.829027), + Vectors.dense(-4.792007,-2.491098,-2.939393,8.086467,3.773812,-9.997300,0.222378,8.995244,-5.753282,6.091060), + Vectors.dense(7.700725,-6.414918,1.684476,-8.983361,4.284580,-9.017608,0.552379,-7.705741,2.589852,0.411561), + Vectors.dense(6.991900,-1.063721,9.321163,-0.429719,-2.167696,-1.736425,-0.919139,6.980681,-0.711914,3.414347), + Vectors.dense(5.794488,-1.062261,0.955322,0.389642,3.012921,-9.953994,-3.197309,3.992421,-6.935902,8.147622), + Vectors.dense(-2.486670,6.973242,-4.047004,-5.655629,5.081786,5.533859,7.821403,2.763313,-0.454056,6.554309), + Vectors.dense(2.204855,7.839522,7.381886,1.618749,-6.566877,7.584285,-8.355983,-5.501410,-8.191205,-2.608499), + Vectors.dense(-9.948613,-8.941953,-8.106389,4.863542,5.852806,-1.659259,6.342504,-8.190106,-3.110330,-7.484658), + )) + + val summary: MultivariateStatisticalSummary = Statistics.colStats(data) + println(summary.mean) // a dense vector containing the mean value for each column + println(summary.variance) // column-wise variance + println(summary.max) + println(summary.min) + println(summary.count) + println(summary.normL1) + println(summary.normL2) + println(summary.weightSum) + println(summary.numNonzeros) // number of nonzeros in each column + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java b/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java new file mode 100644 index 000000000..d34485004 --- /dev/null +++ b/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java @@ -0,0 +1,9 @@ +package org.apache.spark.mllib.stat; + +public class SummarizerResult { + public long meanNumericTable; + public long varianceNumericTable; + public long minimumNumericTable; + public long maximumNumericTable; + +} \ No newline at end of file diff --git a/mllib-dal/src/main/native/Makefile b/mllib-dal/src/main/native/Makefile index 4f18a363b..94a21bba1 100644 --- a/mllib-dal/src/main/native/Makefile +++ b/mllib-dal/src/main/native/Makefile @@ -61,7 +61,8 @@ CPP_SRCS += \ ./ALSDALImpl.cpp ./ALSShuffle.cpp \ ./NaiveBayesDALImpl.cpp \ ./LinearRegressionDALImpl.cpp \ - ./CorrelationDALImpl.cpp + ./CorrelationDALImpl.cpp \ + ./SummarizerDALImpl.cpp OBJS += \ ./OneCCL.o ./OneDAL.o ./service.o ./error_handling.o \ @@ -70,7 +71,8 @@ OBJS += \ ./ALSDALImpl.o ./ALSShuffle.o \ ./NaiveBayesDALImpl.o \ ./LinearRegressionDALImpl.o \ - ./CorrelationDALImpl.o + ./CorrelationDALImpl.o \ + ./SummarizerDALImpl.o DEFINES=-D$(PLATFORM_PROFILE) diff --git a/mllib-dal/src/main/native/SummarizerDALImpl.cpp b/mllib-dal/src/main/native/SummarizerDALImpl.cpp new file mode 100644 index 000000000..6428c2696 --- /dev/null +++ b/mllib-dal/src/main/native/SummarizerDALImpl.cpp @@ -0,0 +1,250 @@ +/******************************************************************************* + * 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. + *******************************************************************************/ + +#include +#include + +#ifdef CPU_GPU_PROFILE +#include "GPU.h" +#endif + +#include "OneCCL.h" +#include "org_apache_spark_mllib_stat_SummarizerDALImpl.h" +#include "service.h" + +using namespace std; +using namespace daal; +using namespace daal::algorithms; + +typedef double algorithmFPType; /* Algorithm floating-point type */ + +static void summarizer_compute(JNIEnv *env, + jobject obj, + int rankId, + ccl::communicator &comm, + const NumericTablePtr &pData, + size_t nBlocks, + jobject resultObj) { + using daal::byte; + auto t1 = std::chrono::high_resolution_clock::now(); + + const bool isRoot = (rankId == ccl_root); + + low_order_moments::Distributed localAlgorithm; + + /* Set the input data set to the algorithm */ + localAlgorithm.input.set(low_order_moments::data, pData); + + /* Compute low_order_moments */ + localAlgorithm.compute(); + + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): local step took " << duration << " secs" + << std::endl; + + t1 = std::chrono::high_resolution_clock::now(); + + /* Serialize partial results required by step 2 */ + services::SharedPtr serializedData; + InputDataArchive dataArch; + localAlgorithm.getPartialResult()->serialize(dataArch); + size_t perNodeArchLength = dataArch.getSizeOfArchive(); + + serializedData = + services::SharedPtr(new byte[perNodeArchLength * nBlocks]); + + byte *nodeResults = new byte[perNodeArchLength]; + dataArch.copyArchiveToArray(nodeResults, perNodeArchLength); + std::vector aReceiveCount(comm.size(), + perNodeArchLength); // 4 x "14016" + + /* Transfer partial results to step 2 on the root node */ + ccl::gather((int8_t *)nodeResults, perNodeArchLength, + (int8_t *)(serializedData.get()), perNodeArchLength, comm) + .wait(); + t2 = std::chrono::high_resolution_clock::now(); + + duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): ccl_gather took " << duration << " secs" + << std::endl; + if (isRoot) { + auto t1 = std::chrono::high_resolution_clock::now(); + /* Create an algorithm to compute covariance on the master node */ + low_order_moments::Distributed masterAlgorithm; + + for (size_t i = 0; i < nBlocks; i++) { + /* Deserialize partial results from step 1 */ + OutputDataArchive dataArch(serializedData.get() + + perNodeArchLength * i, + perNodeArchLength); + + low_order_moments::PartialResultPtr dataForStep2FromStep1(new low_order_moments::PartialResult()); + dataForStep2FromStep1->deserialize(dataArch); + + /* Set local partial results as input for the master-node algorithm + */ + masterAlgorithm.input.add(low_order_moments::partialResults, + dataForStep2FromStep1); + } + + /* Set the parameter to choose the type of the output matrix */ + masterAlgorithm.parameter.estimatesToCompute = low_order_moments::estimatesAll; + + /* Merge and finalizeCompute covariance decomposition on the master node */ + masterAlgorithm.compute(); + masterAlgorithm.finalizeCompute(); + + /* Retrieve the algorithm results */ + low_order_moments::ResultPtr result = masterAlgorithm.getResult(); + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): master step took " << duration << " secs" + << std::endl; + + /* Print the results */ + printNumericTable(result->get(low_order_moments::mean), + "low_order_moments first 20 columns of " + "Mean :", + 1, 20); + printNumericTable(result->get(low_order_moments::variance), + "low_order_moments first 20 columns of " + "Variance :", + 1, 20); + printNumericTable(result->get(low_order_moments::minimum), + "low_order_moments first 20 columns of " + "Minimum :", + 1, 20); + printNumericTable(result->get(low_order_moments::maximum), + "low_order_moments first 20 columns of " + "Maximum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sum), + "low_order_moments first 20 columns of " + "Sum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquares), + "low_order_moments first 20 columns of " + "SumSquares :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquaresCentered), + "low_order_moments first 20 columns of " + "SumSquaresCentered :", + 1, 20); + printNumericTable(result->get(low_order_moments::secondOrderRawMoment), + "low_order_moments first 20 columns of " + "SecondOrderRawMoment :", + 1, 20); + printNumericTable(result->get(low_order_moments::standardDeviation), + "low_order_moments first 20 columns of " + "StandardDeviation :", + 1, 20); + printNumericTable(result->get(low_order_moments::variation), + "low_order_moments first 20 columns of " + "Variation :", + 1, 20); + // Return all covariance & mean + jclass clazz = env->GetObjectClass(resultObj); + + // Get Field references + jfieldID meanNumericTableField = + env->GetFieldID(clazz, "meanNumericTable", "J"); + jfieldID varianceNumericTableField = + env->GetFieldID(clazz, "varianceNumericTable", "J"); + jfieldID minimumNumericTableField = + env->GetFieldID(clazz, "minimumNumericTable", "J"); + jfieldID maximumNumericTableField = + env->GetFieldID(clazz, "maximumNumericTable", "J"); + + NumericTablePtr *mean = + new NumericTablePtr(result->get(low_order_moments::mean)); + NumericTablePtr *variance = + new NumericTablePtr(result->get(low_order_moments::variance)); + NumericTablePtr *max = + new NumericTablePtr(result->get(low_order_moments::maximum)); + NumericTablePtr *min = + new NumericTablePtr(result->get(low_order_moments::minimum)); + + env->SetLongField(resultObj, meanNumericTableField, (jlong)mean); + env->SetLongField(resultObj, varianceNumericTableField, (jlong)variance); + env->SetLongField(resultObj, maximumNumericTableField, (jlong)max); + env->SetLongField(resultObj, minimumNumericTableField, (jlong)min); + + } +} + +/* + * Class: org_apache_spark_mllib_stat_CorrelationDALImpl + * Method: cCorrelationTrainDAL + * Signature: (JJDDIILorg/apache/spark/ml/stat/CorrelationResult;)J + */ +JNIEXPORT jlong JNICALL +Java_org_apache_spark_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL( + JNIEnv *env, jobject obj, jlong pNumTabData, + jint executor_num, jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, jobject resultObj) { + + ccl::communicator &comm = getComm(); + size_t rankId = comm.rank(); + std::cout << " rankId : " << rankId << " ! " + << std::endl; + + const size_t nBlocks = executor_num; + + NumericTablePtr pData = *((NumericTablePtr *)pNumTabData); + + #ifdef CPU_GPU_PROFILE + + if (use_gpu) { + int n_gpu = env->GetArrayLength(gpu_idx_array); + cout << "oneDAL (native): use GPU kernels with " << n_gpu << " GPU(s)" + << endl; + + jint *gpu_indices = env->GetIntArrayElements(gpu_idx_array, 0); + + int size = comm.size(); + auto assigned_gpu = + getAssignedGPU(comm, size, rankId, gpu_indices, n_gpu); + + // Set SYCL context + cl::sycl::queue queue(assigned_gpu); + daal::services::SyclExecutionContext ctx(queue); + daal::services::Environment::getInstance()->setDefaultExecutionContext( + ctx); + + summarizer_compute( + env, obj, rankId, comm, pData, nBlocks, resultObj); + + env->ReleaseIntArrayElements(gpu_idx_array, gpu_indices, 0); + } else + #endif + { + // 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; + + summarizer_compute( + env, obj, rankId, comm, pData, nBlocks, resultObj); + } + + return 0; +} diff --git a/mllib-dal/src/main/native/build-jni.sh b/mllib-dal/src/main/native/build-jni.sh index c58385d40..4191be3c7 100755 --- a/mllib-dal/src/main/native/build-jni.sh +++ b/mllib-dal/src/main/native/build-jni.sh @@ -36,4 +36,5 @@ javah -d $WORK_DIR/javah -classpath "$WORK_DIR/../../../target/classes:$DAAL_JAR org.apache.spark.ml.recommendation.ALSDALImpl \ org.apache.spark.ml.classification.NaiveBayesDALImpl \ org.apache.spark.ml.regression.LinearRegressionDALImpl \ - org.apache.spark.ml.stat.CorrelationDALImpl + org.apache.spark.ml.stat.CorrelationDALImpl \ + org.apache.spark.mllib.stat.SummarizerDALImpl \ No newline at end of file diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h new file mode 100644 index 000000000..4f0681069 --- /dev/null +++ b/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class org_apache_spark_mllib_stat_SummarizerDALImpl */ + +#ifndef _Included_org_apache_spark_mllib_stat_SummarizerDALImpl +#define _Included_org_apache_spark_mllib_stat_SummarizerDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: org_apache_spark_mllib_stat_SummarizerDALImpl + * Method: cSummarizerTrainDAL + * Signature: (JIIZ[ILorg/apache/spark/mllib/stat/SummarizerResult;)J + */ +JNIEXPORT jlong JNICALL Java_org_apache_spark_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL + (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif 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 702d91774..9dec273c3 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 @@ -29,6 +29,10 @@ object Utils { return sc.conf.getBoolean("spark.oap.mllib.enabled", true) } + def isOAPEnabled(sc: SparkContext): Boolean = { + return sc.conf.getBoolean("spark.oap.mllib.enabled", true) + } + def getOneCCLIPPort(data: RDD[_]): String = { val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala new file mode 100644 index 000000000..fb3a9ddfa --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala @@ -0,0 +1,118 @@ +/* + * 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.mllib.stat + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util.Utils.getOneCCLIPPort +import org.apache.spark.ml.util.{OneCCL, OneDAL} +import org.apache.spark.mllib.stat.SummarizerResult +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary => Summary} +import org.apache.spark.rdd.RDD + +class SummarizerDALImpl( + val executorNum: Int, + val executorCores: Int) + extends Serializable with Logging { + + def computeSummarizerMatrix(data: RDD[Vector]): Summary = { + + val kvsIPPort = getOneCCLIPPort(data) + + val sparkContext = data.sparkContext + val useGPU = sparkContext.conf.getBoolean("spark.oap.mllib.useGPU", false) + + val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) + + val results = coalescedTables.mapPartitionsWithIndex { (rank, table) => + + val gpuIndices = if (useGPU) { + val resources = TaskContext.get().resources() + resources("gpu").addresses.map(_.toInt) + } else { + null + } + + val tableArr = table.next() + OneCCL.init(executorNum, rank, kvsIPPort) + + val computeStartTime = System.nanoTime() + + val result = new SummarizerResult() + cSummarizerTrainDAL( + tableArr, + executorNum, + executorCores, + useGPU, + gpuIndices, + result + ) + + val computeEndTime = System.nanoTime() + + val durationCompute = (computeEndTime - computeStartTime).toDouble / 1E9 + + logInfo(s"SummarizerDAL compute took ${durationCompute} secs") + + val ret = if (OneCCL.isRoot()) { + + val convResultStartTime = System.nanoTime() + val meanArray= OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.meanNumericTable)) + val varianceArray = OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.varianceNumericTable)) + val maxrray= OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.maximumNumericTable)) + val minArray = OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.minimumNumericTable)) + + val convResultEndTime = System.nanoTime() + + val durationCovResult = (convResultEndTime - convResultStartTime).toDouble / 1E9 + + logInfo(s"SummarizerDAL result conversion took ${durationCovResult} secs") + + Iterator((meanArray(0), varianceArray(0), maxrray(0), minArray(0))) + } else { + Iterator.empty + } + + OneCCL.cleanup() + + ret + }.collect() + + // Make sure there is only one result from rank 0 + assert(results.length == 1) + + val meanVector = results(0)._1 + val varianceVector = results(0)._2 + val maxVector = results(0)._3 + val minVector = results(0)._4 + + val summary = new MultivariateStatisticalDALSummary(OldVectors.fromML(meanVector), OldVectors.fromML(varianceVector) + , OldVectors.fromML(maxVector), OldVectors.fromML(minVector)) + + summary + } + + + @native private def cSummarizerTrainDAL(data: Long, + executor_num: Int, + executor_cores: Int, + useGPU: Boolean, + gpuIndices: Array[Int], + result: SummarizerResult): Long +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} From 35cec08cb500bf96099eac60e3d047cc3edaf125 Mon Sep 17 00:00:00 2001 From: Xiaochang Wu Date: Mon, 22 Nov 2021 10:08:42 +0800 Subject: [PATCH 02/13] [Core] Refactor and support multiple Spark versions in single JAR (#141) * add prepare-shims-codebase * Add NaiveBayes skeleton, to be revised. * Add NaiveBayes for multi-version * move namespaces * update namespaces * Add PCA shims * NaiveBayes, KMeans, PCA ready * Add ALS shims * Add Shim trait * Fix default paramMap * Add LinearRegression 3.1.1 & 3.1.2 * update prepare-shims-codebase * Update pom to 3.2.0, Add LinearRegression 3.2.0 * Update CI * Update CI * update permission * nit * nit * update ci * update CI * nit * update ci * nit * nit * nit * fix building native change phase process-resources to generate-resources * update CI * update README * nit * Fix spark-submit -v bug for 3.2.0 by removing -v * update apt-get -y -q * update README --- .../{oap-mllib-ci.yml => ci-build.yml} | 11 +- .github/workflows/ci-cluster-test.yml | 28 + .github/workflows/ci-local-test.yml | 28 + .gitignore | 3 + README.md | 22 +- RELEASE | 4 +- conf/env.sh.template | 24 +- dev/ci-build-test.sh | 29 + .../ci-test-cluster.sh => ci-cluster-test.sh} | 23 +- dev/{setup-all.sh => ci-local-test.sh} | 14 +- dev/ci-test.sh | 34 - dev/prepare-build-deps-gpu.sh | 8 + dev/prepare-build-deps.sh | 8 + dev/prepare-shims-codebase.sh | 57 + dev/set-examples-version.sh | 19 - ...setup-spark-envs.sh => load-spark-envs.sh} | 2 +- dev/test-cluster/setup-cluster.sh | 6 +- dev/test-cluster/setup-python3.sh | 14 +- dev/update-release-version.sh | 35 + examples/als-pyspark/run.sh | 2 +- examples/als/pom.xml | 4 +- examples/als/run.sh | 2 +- examples/build-all-scala.sh | 5 + examples/correlation/pom.xml | 4 +- examples/correlation/run-gpu-standalone.sh | 2 +- examples/correlation/run.sh | 2 +- examples/kmeans-pyspark/kmeans-pyspark.py | 2 +- examples/kmeans-pyspark/run.sh | 2 +- examples/kmeans/pom.xml | 4 +- examples/kmeans/run-gpu-standalone.sh | 2 +- examples/kmeans/run.sh | 2 +- examples/linear-regression/pom.xml | 4 +- examples/linear-regression/run.sh | 2 +- examples/naive-bayes/pom.xml | 4 +- examples/naive-bayes/run.sh | 2 +- examples/pca-pyspark/run.sh | 2 +- examples/pca/pom.xml | 4 +- examples/pca/run-gpu-standalone.sh | 2 +- examples/pca/run.sh | 2 +- examples/run-all-pyspark.sh | 5 + examples/run-all-scala.sh | 5 + examples/summarizer/IntelGpuResourceFile.json | 1 + examples/summarizer/build.sh | 3 + examples/summarizer/pom.xml | 94 + examples/summarizer/run-gpu-standalone.sh | 41 + examples/summarizer/run.sh | 30 + .../ml/SummaryStatisticsExample.scala | 67 + mllib-dal/build-cpu-gpu.sh | 98 - mllib-dal/build.sh | 117 +- mllib-dal/pom.xml | 94 +- .../intel/oap/mllib}/CCLParam.java | 2 +- .../intel/oap/mllib}/LibLoader.java | 3 +- .../util => com/intel/oap/mllib}/Service.java | 7 +- .../classification/NaiveBayesResult.java | 2 +- .../oap/mllib}/clustering/KMeansResult.java | 2 +- .../intel/oap/mllib}/feature/PCAResult.java | 2 +- .../recommendation/ALSPartitionInfo.java | 2 +- .../oap/mllib}/recommendation/ALSResult.java | 2 +- .../oap/mllib}/regression/LiRResult.java | 2 +- .../oap/mllib}/stat/CorrelationResult.java | 2 +- .../spark/mllib/stat/SummarizerResult.java | 9 + mllib-dal/src/main/native/ALSDALImpl.cpp | 18 +- .../src/main/native/CorrelationDALImpl.cpp | 9 +- mllib-dal/src/main/native/KMeansDALImpl.cpp | 9 +- .../main/native/LinearRegressionDALImpl.cpp | 9 +- mllib-dal/src/main/native/Makefile | 6 +- .../src/main/native/NaiveBayesDALImpl.cpp | 9 +- mllib-dal/src/main/native/OneCCL.cpp | 44 +- mllib-dal/src/main/native/OneDAL.cpp | 66 +- mllib-dal/src/main/native/PCADALImpl.cpp | 10 +- .../src/main/native/SummarizerDALImpl.cpp | 250 +++ mllib-dal/src/main/native/build-jni.sh | 14 +- .../javah/com_intel_oap_mllib_OneCCL__.h | 61 + .../javah/com_intel_oap_mllib_OneDAL__.h | 69 + ...p_mllib_classification_NaiveBayesDALImpl.h | 21 + ...intel_oap_mllib_clustering_KMeansDALImpl.h | 21 + .../com_intel_oap_mllib_feature_PCADALImpl.h | 21 + ...ntel_oap_mllib_recommendation_ALSDALImpl.h | 29 + ...mllib_regression_LinearRegressionDALImpl.h | 21 + ..._intel_oap_mllib_stat_CorrelationDALImpl.h | 21 + ...park_ml_classification_NaiveBayesDALImpl.h | 21 - ...apache_spark_ml_clustering_KMeansDALImpl.h | 21 - .../org_apache_spark_ml_feature_PCADALImpl.h | 21 - ...pache_spark_ml_recommendation_ALSDALImpl.h | 29 - ...rk_ml_regression_LinearRegressionDALImpl.h | 21 - ..._apache_spark_ml_stat_CorrelationDALImpl.h | 21 - .../javah/org_apache_spark_ml_util_OneCCL__.h | 61 - .../javah/org_apache_spark_ml_util_OneDAL__.h | 69 - ...pache_spark_mllib_stat_SummarizerDALImpl.h | 21 + mllib-dal/src/main/resources/log4j.properties | 2 +- .../util => com/intel/oap/mllib}/OneCCL.scala | 13 +- .../util => com/intel/oap/mllib}/OneDAL.scala | 10 +- .../util => com/intel/oap/mllib}/Utils.scala | 28 +- .../classification/NaiveBayesDALImpl.scala | 23 +- .../mllib/classification/NaiveBayesShim.scala | 40 + .../oap/mllib}/clustering/KMeansDALImpl.scala | 13 +- .../oap/mllib/clustering/KMeansShim.scala | 40 + .../intel/oap/mllib}/feature/PCADALImpl.scala | 44 +- .../com/intel/oap/mllib/feature/PCAShim.scala | 40 + .../mllib}/recommendation/ALSDALImpl.scala | 13 +- .../oap/mllib/recommendation/ALSShim.scala | 56 + .../regression/LinearRegressionDALImpl.scala | 18 +- .../regression/LinearRegressionShim.scala | 42 + .../oap/mllib}/stat/CorrelationDALImpl.scala | 11 +- .../oap/mllib/stat/CorrelationShim.scala | 43 + .../spark/ml/classification/NaiveBayes.scala | 130 ++ .../classification/spark320/NaiveBayes.scala | 332 +++ .../apache/spark/ml/clustering/KMeans.scala | 109 + .../spark/ml/clustering/spark320/KMeans.scala | 189 ++ .../org/apache/spark/ml/feature/PCA.scala | 76 + .../spark/ml/feature/spark320/PCA.scala | 80 + .../apache/spark/ml/recommendation/ALS.scala | 741 +------ .../ml/recommendation/spark312}/ALS.scala | 760 +------ .../ml/recommendation/spark320}/ALS.scala | 739 +------ .../ml/regression/LinearRegression.scala | 271 +++ .../regression/spark312/HuberAggregator.scala | 250 +++ .../spark312/LeastSquaresAggregator.scala | 314 +++ .../spark312}/LinearRegression.scala | 980 ++++----- .../spark320/LinearRegression.scala | 1116 ++++++++++ .../apache/spark/ml/stat/Correlation.scala | 42 +- .../spark/ml/stat/spark320}/Correlation.scala | 13 +- .../spark/mllib/clustering/KMeans.scala | 0 .../spark/mllib/stat/SummarizerDALImpl.scala | 118 ++ .../spark/ml/classification/NaiveBayes.scala | 677 ------ .../apache/spark/ml/clustering/KMeans.scala | 507 ----- .../org/apache/spark/ml/feature/PCA.scala | 255 --- .../ml/regression/LinearRegression.scala | 1065 ---------- .../apache/spark/ml/stat/Correlation.scala | 111 - .../spark/mllib/clustering/KMeans.scala | 523 ----- .../MultivariateStatisticalDALSummary.scala | 65 + .../apache/spark/mllib/stat/Statistics.scala | 275 +++ .../spark/ml/classification/NaiveBayes.scala | 675 ------ .../apache/spark/ml/clustering/KMeans.scala | 494 ----- .../org/apache/spark/ml/feature/PCA.scala | 255 --- .../ml/regression/LinearRegression.scala | 1059 --------- .../apache/spark/ml/stat/Correlation.scala | 111 - .../spark/mllib/clustering/KMeans.scala | 520 ----- .../MultivariateStatisticalDALSummary.scala | 65 + .../apache/spark/mllib/stat/Statistics.scala | 275 +++ .../spark/ml/classification/NaiveBayes.scala | 675 ------ .../apache/spark/ml/clustering/KMeans.scala | 493 ----- .../org/apache/spark/ml/feature/PCA.scala | 255 --- .../ml/regression/LinearRegression.scala | 1059 --------- .../spark/mllib/clustering/KMeans.scala | 520 ----- .../MultivariateStatisticalDALSummary.scala | 65 + .../apache/spark/mllib/stat/Statistics.scala | 275 +++ .../spark/ml/classification/NaiveBayes.scala | 684 ------ .../apache/spark/ml/clustering/KMeans.scala | 492 ----- .../org/apache/spark/ml/feature/PCA.scala | 255 --- .../apache/spark/ml/recommendation/ALS.scala | 1885 ----------------- .../MultivariateStatisticalDALSummary.scala | 65 + .../apache/spark/mllib/stat/Statistics.scala | 275 +++ mllib-dal/src/test/resources/log4j.properties | 2 +- .../spark/ml/feature/MLlibPCASuite.scala | 2 +- mllib-dal/test-cpu-gpu.sh | 119 -- mllib-dal/test.sh | 85 +- 156 files changed, 6562 insertions(+), 16279 deletions(-) rename .github/workflows/{oap-mllib-ci.yml => ci-build.yml} (65%) create mode 100644 .github/workflows/ci-cluster-test.yml create mode 100644 .github/workflows/ci-local-test.yml create mode 100755 dev/ci-build-test.sh rename dev/{test-cluster/ci-test-cluster.sh => ci-cluster-test.sh} (53%) rename dev/{setup-all.sh => ci-local-test.sh} (61%) delete mode 100755 dev/ci-test.sh create mode 100755 dev/prepare-shims-codebase.sh delete mode 100755 dev/set-examples-version.sh rename dev/test-cluster/{setup-spark-envs.sh => load-spark-envs.sh} (95%) create mode 100755 dev/update-release-version.sh create mode 100644 examples/summarizer/IntelGpuResourceFile.json create mode 100644 examples/summarizer/build.sh create mode 100644 examples/summarizer/pom.xml create mode 100755 examples/summarizer/run-gpu-standalone.sh create mode 100644 examples/summarizer/run.sh create mode 100644 examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala delete mode 100755 mllib-dal/build-cpu-gpu.sh rename mllib-dal/src/main/java/{org/apache/spark/ml/util => com/intel/oap/mllib}/CCLParam.java (95%) rename mllib-dal/src/main/java/{org/apache/spark/ml/util => com/intel/oap/mllib}/LibLoader.java (98%) rename mllib-dal/src/main/java/{org/apache/spark/ml/util => com/intel/oap/mllib}/Service.java (98%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/classification/NaiveBayesResult.java (69%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/clustering/KMeansResult.java (94%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/feature/PCAResult.java (95%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/recommendation/ALSPartitionInfo.java (93%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/recommendation/ALSResult.java (94%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/regression/LiRResult.java (95%) rename mllib-dal/src/main/java/{org/apache/spark/ml => com/intel/oap/mllib}/stat/CorrelationResult.java (96%) create mode 100644 mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java create mode 100644 mllib-dal/src/main/native/SummarizerDALImpl.cpp create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneCCL__.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneDAL__.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_classification_NaiveBayesDALImpl.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_clustering_KMeansDALImpl.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_feature_PCADALImpl.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_recommendation_ALSDALImpl.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_regression_LinearRegressionDALImpl.h create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_CorrelationDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_classification_NaiveBayesDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_clustering_KMeansDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_feature_PCADALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_recommendation_ALSDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_regression_LinearRegressionDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_stat_CorrelationDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneCCL__.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h create mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h rename mllib-dal/src/main/scala/{org/apache/spark/ml/util => com/intel/oap/mllib}/OneCCL.scala (92%) rename mllib-dal/src/main/scala/{org/apache/spark/ml/util => com/intel/oap/mllib}/OneDAL.scala (99%) rename mllib-dal/src/main/scala/{org/apache/spark/ml/util => com/intel/oap/mllib}/Utils.scala (83%) rename mllib-dal/src/main/scala/{org/apache/spark/ml => com/intel/oap/mllib}/classification/NaiveBayesDALImpl.scala (84%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesShim.scala rename mllib-dal/src/main/scala/{org/apache/spark/ml => com/intel/oap/mllib}/clustering/KMeansDALImpl.scala (88%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansShim.scala rename mllib-dal/src/main/scala/{org/apache/spark/ml => com/intel/oap/mllib}/feature/PCADALImpl.scala (81%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCAShim.scala rename mllib-dal/src/main/scala/{org/apache/spark/ml => com/intel/oap/mllib}/recommendation/ALSDALImpl.scala (98%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSShim.scala rename mllib-dal/src/main/scala/{org/apache/spark/ml => com/intel/oap/mllib}/regression/LinearRegressionDALImpl.scala (87%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionShim.scala rename mllib-dal/src/main/scala/{org/apache/spark/ml => com/intel/oap/mllib}/stat/CorrelationDALImpl.scala (90%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationShim.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/classification/spark320/NaiveBayes.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/clustering/spark320/KMeans.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCA.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/feature/spark320/PCA.scala rename mllib-dal/src/{spark-3.0.2 => }/main/scala/org/apache/spark/ml/recommendation/ALS.scala (60%) rename mllib-dal/src/{spark-3.0.0/main/scala/org/apache/spark/ml/recommendation => main/scala/org/apache/spark/ml/recommendation/spark312}/ALS.scala (60%) rename mllib-dal/src/{spark-3.0.1/main/scala/org/apache/spark/ml/recommendation => main/scala/org/apache/spark/ml/recommendation/spark320}/ALS.scala (62%) create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/HuberAggregator.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LeastSquaresAggregator.scala rename mllib-dal/src/{spark-3.1.1/main/scala/org/apache/spark/ml/regression => main/scala/org/apache/spark/ml/regression/spark312}/LinearRegression.scala (58%) create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark320/LinearRegression.scala rename mllib-dal/src/{spark-3.0.2 => }/main/scala/org/apache/spark/ml/stat/Correlation.scala (69%) rename mllib-dal/src/{spark-3.1.1/main/scala/org/apache/spark/ml/stat => main/scala/org/apache/spark/ml/stat/spark320}/Correlation.scala (95%) rename mllib-dal/src/{spark-3.1.1 => }/main/scala/org/apache/spark/mllib/clustering/KMeans.scala (100%) create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/clustering/KMeans.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/feature/PCA.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/regression/LinearRegression.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/stat/Correlation.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/clustering/KMeans.scala create mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/feature/PCA.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/stat/Correlation.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/clustering/KMeans.scala create mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/clustering/KMeans.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/feature/PCA.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/regression/LinearRegression.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/clustering/KMeans.scala create mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala delete mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala delete mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/feature/PCA.scala delete mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala create mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100755 mllib-dal/test-cpu-gpu.sh diff --git a/.github/workflows/oap-mllib-ci.yml b/.github/workflows/ci-build.yml similarity index 65% rename from .github/workflows/oap-mllib-ci.yml rename to .github/workflows/ci-build.yml index 1b28d0a79..5dfbf6624 100644 --- a/.github/workflows/oap-mllib-ci.yml +++ b/.github/workflows/ci-build.yml @@ -1,4 +1,4 @@ -name: OAP MLlib CI +name: OAP MLlib Build Test on: [push, pull_request] @@ -20,12 +20,9 @@ jobs: ~/.m2/repository /opt/intel/oneapi ~/opt - key: ${{ runner.os }}_spark-3.1.1_hadoop-3.2.0_oneapi-2021.4.0 + key: ${{ runner.os }}_spark-3.2.0_hadoop-3.2.0_oneapi-2021.4.0 restore-keys: | ${{ runner.os }}- - - name: Set up environments - run: | - source ${{github.workspace}}/dev/setup-all.sh - - name: Build and Test + - name: Build Test for CPU and GPU run: | - ${{github.workspace}}/dev/ci-test.sh + ${{github.workspace}}/dev/ci-build-test.sh diff --git a/.github/workflows/ci-cluster-test.yml b/.github/workflows/ci-cluster-test.yml new file mode 100644 index 000000000..ecf5d46cb --- /dev/null +++ b/.github/workflows/ci-cluster-test.yml @@ -0,0 +1,28 @@ +name: OAP MLlib Cluster Test + +on: [push, pull_request] + +jobs: + build: + runs-on: ubuntu-18.04 + + steps: + - uses: actions/checkout@v2 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Restore cached dependencies + uses: actions/cache@v2 + with: + path: | + # /var/cache/apt/archives/*.deb + ~/.m2/repository + /opt/intel/oneapi + ~/opt + key: ${{ runner.os }}_spark-3.2.0_hadoop-3.2.0_oneapi-2021.4.0 + restore-keys: | + ${{ runner.os }}- + - name: Cluster Test + run: | + ${{github.workspace}}/dev/ci-cluster-test.sh diff --git a/.github/workflows/ci-local-test.yml b/.github/workflows/ci-local-test.yml new file mode 100644 index 000000000..93f8f1c06 --- /dev/null +++ b/.github/workflows/ci-local-test.yml @@ -0,0 +1,28 @@ +name: OAP MLlib Local Test + +on: [push, pull_request] + +jobs: + build: + runs-on: ubuntu-18.04 + + steps: + - uses: actions/checkout@v2 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Restore cached dependencies + uses: actions/cache@v2 + with: + path: | + # /var/cache/apt/archives/*.deb + ~/.m2/repository + /opt/intel/oneapi + ~/opt + key: ${{ runner.os }}_spark-3.2.0_hadoop-3.2.0_oneapi-2021.4.0 + restore-keys: | + ${{ runner.os }}- + - name: Local Test + run: | + ${{github.workspace}}/dev/ci-local-test.sh diff --git a/.gitignore b/.gitignore index b69b6d7f3..b4f4030d5 100644 --- a/.gitignore +++ b/.gitignore @@ -5,3 +5,6 @@ target/ .idea/ .idea_modules/ +*.versionsBackup +shims-sources/ +env.sh \ No newline at end of file diff --git a/README.md b/README.md index 0667dc69c..ecac590ee 100644 --- a/README.md +++ b/README.md @@ -24,7 +24,7 @@ You can find the all the OAP MLlib documents on the [project web page](https://o ### Java/Scala Users Preferred -Use a pre-built OAP MLlib JAR to get started. You can firstly download OAP package from [OAP-JARs-Tarball](https://github.com/oap-project/oap-tools/releases/download/v1.2.0/oap-1.2.0-bin.tar.gz) and extract this Tarball to get `oap-mllib-x.x.x.jar` under `oap-1.2.0-bin-spark-3.1.1/jars`. +Use a pre-built OAP MLlib JAR to get started. You can firstly download OAP package from [OAP-JARs-Tarball](https://github.com/oap-project/oap-tools/releases/download/v1.2.0/oap-1.2.0-bin.tar.gz) and extract this Tarball to get `oap-mllib-x.x.x.jar` under `oap-x.x.x-bin-spark-x.x.x/jars`. Then you can refer to the following [Running](#running) section to try out. @@ -44,7 +44,7 @@ You can also build the package from source code, please refer to [Building](#bui * CentOS 7.0+, Ubuntu 18.04 LTS+ * Java JRE 8.0+ Runtime -* Apache Spark 3.0.0+ +* Apache Spark 3.1.1, 3.1.2 and 3.2.0 Generally, our common system requirements are the same with Intel® oneAPI Toolkit, please refer to [here](https://software.intel.com/content/www/us/en/develop/articles/intel-oneapi-base-toolkit-system-requirements.html) for details. @@ -94,8 +94,7 @@ Edit related variables in "`Minimun Settings`" of `env.sh` #### Upload example data files to HDFS ```bash $ cd examples - $ hadoop fs -mkdir -p /user/$USER - $ hadoop fs -copyFromLocal data + $ hadoop fs -copyFromLocal data / $ hadoop fs -ls data ``` #### Run K-means @@ -123,13 +122,14 @@ We use [Apache Maven](https://maven.apache.org/) to manage and build source code - DPC++/C++ Compiler (dpcpp/clang++) - Data Analytics Library (oneDAL) - Threading Building Blocks (oneTBB) + - MPI Library (MPI) - Collective Communications Library (oneCCL)] Generally you only need to install __Intel® oneAPI Base Toolkit for Linux__ with all or selected components mentioned above. Intel® oneAPI Base Toolkit can be downloaded and installed from [here](https://software.intel.com/content/www/us/en/develop/tools/oneapi.html). Installation process for oneAPI using Package Managers (YUM (DNF), APT, and ZYPPER) is also available. More details about oneAPI can be found [here](https://software.intel.com/content/www/us/en/develop/tools/oneapi.html). Scala and Java dependency descriptions are already included in Maven POM file. -***Note:*** You can refer to [this script](dev/install-build-deps-centos.sh) to install correct dependencies: DPC++/C++, oneDAL, oneTBB, oneCCL. +***Note:*** You can refer to [this script](dev/install-build-deps-centos.sh) to install correct dependencies. ### Build @@ -149,6 +149,7 @@ Environment | Description JAVA_HOME | Path to JDK home directory DAALROOT | Path to oneDAL home directory TBB_ROOT | Path to oneTBB home directory +I_MPI_ROOT | Path to Intel MPI home directory CCL_ROOT | Path to oneCCL home directory We suggest you to source `setvars.sh` script into current shell to setup building environments as following: @@ -164,14 +165,10 @@ If you prefer to buid your own open source [oneDAL](https://github.com/oneapi-sr To build, run the following commands: ```bash $ cd mllib-dal + $ ../dev/prepare-build-deps.sh $ ./build.sh ``` -If no parameter is given, the Spark version __3.1.1__ will be activated by default. You can also specify a different Spark version with option `-p spark-x.x.x`. For example: -```bash - $ ./build.sh -p spark-3.0.0 -``` - The built JAR package will be placed in `target` directory with the name `oap-mllib-x.x.x.jar`. ## Examples @@ -201,7 +198,8 @@ Algorithm | CPU | GPU | Maturity ------------------|-----|-----|--------- K-Means | X | X | Stable PCA | X | X | Stable -ALS | X | | Experimental +ALS | X | | Stable Naive Bayes | X | | Stable -Linear Regression | X | | Stable +Linear Regression | X | | Experimental +Ridge Regression | X | | Experimental Correlation | X | X | Experimental diff --git a/RELEASE b/RELEASE index a72a32503..7e9ffe4e3 100644 --- a/RELEASE +++ b/RELEASE @@ -1 +1,3 @@ -OAP_MLLIB_VERSION=1.2.0 \ No newline at end of file +OAP_MLLIB_VERSION=1.3.0 +SPARK_VERSION=3.2.0 +PLATFORM_PROFILE=CPU_ONLY_PROFILE \ No newline at end of file diff --git a/conf/env.sh.template b/conf/env.sh.template index 168f9d133..7eed7b84c 100644 --- a/conf/env.sh.template +++ b/conf/env.sh.template @@ -2,16 +2,26 @@ # ============== Minimum Settings ============= # -# Set Spark master -SPARK_MASTER=yarn +# Set Spark master, should be spark://xxx or yarn +SPARK_MASTER=spark://localhost:7077 +# SPARK_MASTER=yarn + # Set Hadoop home path -export HADOOP_HOME=/path/to/your/hadoop/home +export HADOOP_HOME=$HADOOP_HOME +# export HADOOP_HOME=/path/to/your/hadoop/home + # Set Spark home path -export SPARK_HOME=/path/to/your/spark/home -# Set HDFS Root, should be hdfs://xxx or file://xxx -export HDFS_ROOT=hdfs://localhost:8020 +export SPARK_HOME=$SPARK_HOME +# export SPARK_HOME=/path/to/your/spark/home + # Set OAP MLlib source code root directory -export OAP_MLLIB_ROOT=/path/to/oap-mllib/home +SCRIPT_DIR=$( cd $(dirname ${BASH_SOURCE[0]}) && pwd ) +export OAP_MLLIB_ROOT=$(cd $SCRIPT_DIR/.. && pwd) +# export OAP_MLLIB_ROOT=/path/to/oap-mllib/home + +# Set HDFS Root, should be hdfs://xxx or file://xxx +export HDFS_ROOT=file://$OAP_MLLIB_ROOT/examples +# export HDFS_ROOT=hdfs://localhost:8020 # ============================================= # diff --git a/dev/ci-build-test.sh b/dev/ci-build-test.sh new file mode 100755 index 000000000..566c974b4 --- /dev/null +++ b/dev/ci-build-test.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +# exit when any command fails +set -e + +# keep track of the last executed command +trap 'last_command=$current_command; current_command=$BASH_COMMAND' DEBUG +# echo an error message before exiting +trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT + +# Install dependencies for building +$GITHUB_WORKSPACE/dev/install-build-deps-ubuntu.sh + +# Setup building envs +source /opt/intel/oneapi/setvars.sh + +# +# Build test for CPU +# +cd $GITHUB_WORKSPACE/mllib-dal +../dev/prepare-build-deps.sh +./build.sh -p CPU_ONLY_PROFILE -q + +# +# Build test for GPU +# +cd $GITHUB_WORKSPACE/mllib-dal +../dev/prepare-build-deps-gpu.sh +./build.sh -p CPU_GPU_PROFILE -q diff --git a/dev/test-cluster/ci-test-cluster.sh b/dev/ci-cluster-test.sh similarity index 53% rename from dev/test-cluster/ci-test-cluster.sh rename to dev/ci-cluster-test.sh index d86d89aef..3bc96a7ff 100755 --- a/dev/test-cluster/ci-test-cluster.sh +++ b/dev/ci-cluster-test.sh @@ -8,21 +8,32 @@ trap 'last_command=$current_command; current_command=$BASH_COMMAND' DEBUG # echo an error message before exiting trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT -# Setup Spark envs -source $GITHUB_WORKSPACE/dev/test-cluster/setup-spark-envs.sh +# Install dependencies for building +$GITHUB_WORKSPACE/dev/install-build-deps-ubuntu.sh + +# Setup building envs +source /opt/intel/oneapi/setvars.sh + +# Prepare lib resources +cd $GITHUB_WORKSPACE/mllib-dal +../dev/prepare-build-deps.sh +./build.sh -p CPU_ONLY_PROFILE -q + +# Setup cluster +source $GITHUB_WORKSPACE/dev/test-cluster/setup-cluster.sh # Setup OAP MLlib envs cp $GITHUB_WORKSPACE/dev/test-cluster/env.sh $GITHUB_WORKSPACE/conf - cd $GITHUB_WORKSPACE/examples -HOST_NAME=$(hostname -f) -export HDFS_ROOT=hdfs://$HOST_NAME:8020 - # Copy examples data to HDFS hadoop fs -copyFromLocal data / hadoop fs -find / +echo "=========================================" +echo "Cluster Testing with Spark Version: $SPARK_VERSION" +echo "=========================================" + # Build and run all examples ./build-all-scala.sh ./run-all-scala.sh diff --git a/dev/setup-all.sh b/dev/ci-local-test.sh similarity index 61% rename from dev/setup-all.sh rename to dev/ci-local-test.sh index 7c08ce0e4..aea7cc6ae 100755 --- a/dev/setup-all.sh +++ b/dev/ci-local-test.sh @@ -11,9 +11,13 @@ trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT # Install dependencies for building $GITHUB_WORKSPACE/dev/install-build-deps-ubuntu.sh -# Setup password-less & python3 -$GITHUB_WORKSPACE/dev/test-cluster/config-ssh.sh -$GITHUB_WORKSPACE/dev/test-cluster/setup-python3.sh +# Setup building envs +source /opt/intel/oneapi/setvars.sh -# Setup cluster and envs -source $GITHUB_WORKSPACE/dev/test-cluster/setup-cluster.sh +# Prepare lib resources +cd $GITHUB_WORKSPACE/mllib-dal +../dev/prepare-build-deps.sh +./build.sh -p CPU_ONLY_PROFILE -q + +unset LD_LIBRARY_PATH +./test.sh -p CPU_ONLY_PROFILE -q diff --git a/dev/ci-test.sh b/dev/ci-test.sh deleted file mode 100755 index 59c64eb7d..000000000 --- a/dev/ci-test.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash - -# exit when any command fails -set -e - -# keep track of the last executed command -trap 'last_command=$current_command; current_command=$BASH_COMMAND' DEBUG -# echo an error message before exiting -trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT - -# Setup building envs -source /opt/intel/oneapi/setvars.sh - -# Prepare lib resources -cd $GITHUB_WORKSPACE/mllib-dal -../dev/prepare-build-deps.sh - -# Test for all versions -SupportedSparkVersions=("spark-3.0.0" "spark-3.0.1" "spark-3.0.2" "spark-3.1.1") -for SparkVer in ${SupportedSparkVersions[*]}; do - echo - echo "========================================" - echo "Testing with Spark Version: $SparkVer" - echo "========================================" - echo - cd $GITHUB_WORKSPACE/mllib-dal - ./build.sh -q - ./test.sh -q -p $SparkVer -done - -# Yarn cluster test with default spark version -cd $GITHUB_WORKSPACE/mllib-dal -./build.sh -q -$GITHUB_WORKSPACE/dev/test-cluster/ci-test-cluster.sh diff --git a/dev/prepare-build-deps-gpu.sh b/dev/prepare-build-deps-gpu.sh index e6762e1c7..abea6fd0f 100755 --- a/dev/prepare-build-deps-gpu.sh +++ b/dev/prepare-build-deps-gpu.sh @@ -1,5 +1,13 @@ #!/usr/bin/env bash +# exit when any command fails +set -e + +# keep track of the last executed command +trap 'last_command=$current_command; current_command=$BASH_COMMAND' DEBUG +# echo an error message before exiting +trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT + if [ -z ${ONEAPI_ROOT} ]; then echo Please source Intel oneAPI Toolkit environments! exit 1 diff --git a/dev/prepare-build-deps.sh b/dev/prepare-build-deps.sh index 6b74dfed2..b3bbea9c7 100755 --- a/dev/prepare-build-deps.sh +++ b/dev/prepare-build-deps.sh @@ -1,5 +1,13 @@ #!/usr/bin/env bash +# exit when any command fails +set -e + +# keep track of the last executed command +trap 'last_command=$current_command; current_command=$BASH_COMMAND' DEBUG +# echo an error message before exiting +trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT + if [ -z ${ONEAPI_ROOT} ]; then echo Please source Intel oneAPI Toolkit environments! exit 1 diff --git a/dev/prepare-shims-codebase.sh b/dev/prepare-shims-codebase.sh new file mode 100755 index 000000000..8f876aebb --- /dev/null +++ b/dev/prepare-shims-codebase.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash + +if [[ ! $# -eq 1 ]]; then + echo + echo "Usage: ./prepare-shim-codebash.sh " + echo + exit 0 +fi + +SPARK_VERSIONS=( + 3.1.1 + 3.1.2 + 3.2.0 +) + +FILES=( + org/apache/spark/ml/classification/NaiveBayes.scala + org/apache/spark/ml/clustering/KMeans.scala + org/apache/spark/ml/feature/PCA.scala + org/apache/spark/ml/recommendation/ALS.scala + org/apache/spark/ml/regression/LinearRegression.scala + org/apache/spark/ml/stat/Correlation.scala + org/apache/spark/ml/optim/aggregator/HuberAggregator.scala + org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala + org/apache/spark/mllib/clustering/KMeans.scala +) + +SCRIPT_DIR="$(cd $(dirname ${BASH_SOURCE[0]}) && pwd)" + +SPARK_SOURCE_DIR=$1 +SOURCE_DIR=$SPARK_SOURCE_DIR/mllib/src/main/scala +TARGET_DIR=$SCRIPT_DIR/shims-sources + +copy_sources() { + VERSION=$1 + echo ========================== + echo Checking out v$VERSION ... + git checkout v$VERSION + git status + echo + echo Copying sources for v$VERSION ... + for file in ${FILES[*]}; do + rsync -av --relative $SOURCE_DIR/./$file $TARGET_DIR/$VERSION + done + echo +} + +[ -d $TARGET_DIR ] || mkdir $TARGET_DIR + +cd $SPARK_SOURCE_DIR + +for VERSION in ${SPARK_VERSIONS[*]}; do + mkdir -p $TARGET_DIR/$VERSION + copy_sources $VERSION +done + +[[ -n $(which tree) ]] && tree $TARGET_DIR diff --git a/dev/set-examples-version.sh b/dev/set-examples-version.sh deleted file mode 100755 index a54fae74d..000000000 --- a/dev/set-examples-version.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash - -SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" - -SPARK_VERSION=3.1.1 -OAP_VERSION=1.2.0 -OAP_EXAMPLE_VERSION=$OAP_VERSION - -exampleDirs=(kmeans pca als naive-bayes linear-regression) - -cd $SCRIPT_DIR/../examples - -for dir in ${exampleDirs[*]} -do - cd $dir - mvn versions:set -DnewVersion=$OAP_EXAMPLE_VERSION - mvn versions:set-property -Dproperty=spark.version -DnewVersion=$SPARK_VERSION - cd .. -done diff --git a/dev/test-cluster/setup-spark-envs.sh b/dev/test-cluster/load-spark-envs.sh similarity index 95% rename from dev/test-cluster/setup-spark-envs.sh rename to dev/test-cluster/load-spark-envs.sh index 5e988c3a9..3956cb737 100755 --- a/dev/test-cluster/setup-spark-envs.sh +++ b/dev/test-cluster/load-spark-envs.sh @@ -3,7 +3,7 @@ set -x HADOOP_VERSION=3.2.0 -SPARK_VERSION=3.1.1 +SPARK_VERSION=3.2.0 SPARK_HADOOP_VERSION=hadoop3.2 export HADOOP_HOME=~/opt/hadoop-$HADOOP_VERSION diff --git a/dev/test-cluster/setup-cluster.sh b/dev/test-cluster/setup-cluster.sh index a5b48490e..3d45d9916 100755 --- a/dev/test-cluster/setup-cluster.sh +++ b/dev/test-cluster/setup-cluster.sh @@ -12,8 +12,12 @@ SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" echo JAVA_HOME is $JAVA_HOME +# Setup password-less & python3 +$GITHUB_WORKSPACE/dev/test-cluster/config-ssh.sh +$GITHUB_WORKSPACE/dev/test-cluster/setup-python3.sh + # setup envs -source $SCRIPT_DIR/setup-spark-envs.sh +source $SCRIPT_DIR/load-spark-envs.sh # download spark & hadoop bins [ -d ~/opt ] || mkdir ~/opt diff --git a/dev/test-cluster/setup-python3.sh b/dev/test-cluster/setup-python3.sh index 29208dc5e..4aeedd41e 100755 --- a/dev/test-cluster/setup-python3.sh +++ b/dev/test-cluster/setup-python3.sh @@ -1,12 +1,14 @@ #!/usr/bin/env bash -sudo apt-get update -sudo apt-get install python3-pip python3-setuptools python3-wheel +sudo apt-get update -q +sudo apt-get install -y -q python3-pip python3-setuptools python3-wheel -pip3 install --user numpy +pip3 install --user numpy -q -echo python is in $(which python) +sudo update-alternatives --install /usr/bin/python python /usr/bin/python3 10 + +echo python is in $(which python) python --version -echo python3 is in $(which python3) -python3 --version +echo python3 is in $(which python3) +python3 --version \ No newline at end of file diff --git a/dev/update-release-version.sh b/dev/update-release-version.sh new file mode 100755 index 000000000..94e2bf5d9 --- /dev/null +++ b/dev/update-release-version.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash + +set -e + +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" + +[[ -f $SCRIPT_DIR/../RELEASE ]] && source $SCRIPT_DIR/../RELEASE + +if [[ -z $OAP_MLLIB_VERSION ]]; then + echo OAP_MLLIB_VERSION not defined in RELEASE! + exit 1 +fi + +if [[ -z $SPARK_VERSION ]]; then + echo SPARK_VERSION not defined in RELEASE! + exit 1 +fi + +echo Updating release version to OAP MLlib $OAP_MLLIB_VERSION for Spark $SPARK_VERSION ... + +cd $SCRIPT_DIR/../mllib-dal +mvn versions:set -DnewVersion=$OAP_MLLIB_VERSION +mvn versions:set-property -Dproperty=spark.version -DnewVersion=$SPARK_VERSION + +exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) + +cd $SCRIPT_DIR/../examples + +for dir in ${exampleDirs[*]} +do + cd $dir + mvn versions:set -DnewVersion=$OAP_MLLIB_VERSION + mvn versions:set-property -Dproperty=spark.version -DnewVersion=$SPARK_VERSION + cd .. +done diff --git a/examples/als-pyspark/run.sh b/examples/als-pyspark/run.sh index c8ec93ba4..b85ce26e9 100755 --- a/examples/als-pyspark/run.sh +++ b/examples/als-pyspark/run.sh @@ -8,7 +8,7 @@ DATA_FILE=$HDFS_ROOT/data/onedal_als_csr_ratings.txt APP_PY=als-pyspark.py -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/als/pom.xml b/examples/als/pom.xml index b4e03f26f..c064d6717 100644 --- a/examples/als/pom.xml +++ b/examples/als/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar ALSExample @@ -15,7 +15,7 @@ 1.2.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/examples/als/run.sh b/examples/als/run.sh index ba360ba08..cbb3ce34f 100755 --- a/examples/als/run.sh +++ b/examples/als/run.sh @@ -9,7 +9,7 @@ DATA_FILE=$HDFS_ROOT/data/onedal_als_csr_ratings.txt APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar APP_CLASS=org.apache.spark.examples.ml.ALSExample -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/build-all-scala.sh b/examples/build-all-scala.sh index ee5aa64e3..883f87868 100755 --- a/examples/build-all-scala.sh +++ b/examples/build-all-scala.sh @@ -5,6 +5,11 @@ exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) for dir in ${exampleDirs[*]} do cd $dir + echo + echo =================== + echo Building $dir ... + echo =================== + echo ./build.sh cd .. done diff --git a/examples/correlation/pom.xml b/examples/correlation/pom.xml index 40b64af2d..ae9092fb9 100644 --- a/examples/correlation/pom.xml +++ b/examples/correlation/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar CorrelationExample @@ -15,7 +15,7 @@ 1.2.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/examples/correlation/run-gpu-standalone.sh b/examples/correlation/run-gpu-standalone.sh index b5a432398..39125956f 100755 --- a/examples/correlation/run-gpu-standalone.sh +++ b/examples/correlation/run-gpu-standalone.sh @@ -16,7 +16,7 @@ EXECUTOR_GPU_AMOUNT=1 TASK_GPU_AMOUNT=1 # Should run in standalone mode -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/correlation/run.sh b/examples/correlation/run.sh index 43b7c3300..cea8e50eb 100755 --- a/examples/correlation/run.sh +++ b/examples/correlation/run.sh @@ -6,7 +6,7 @@ source ../../conf/env.sh APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar APP_CLASS=org.apache.spark.examples.ml.CorrelationExample -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/kmeans-pyspark/kmeans-pyspark.py b/examples/kmeans-pyspark/kmeans-pyspark.py index cf93e6034..8634dd676 100644 --- a/examples/kmeans-pyspark/kmeans-pyspark.py +++ b/examples/kmeans-pyspark/kmeans-pyspark.py @@ -39,7 +39,7 @@ .getOrCreate() if (len(sys.argv) != 2) : - println("Require data file path as input parameter") + print("Require data file path as input parameter") sys.exit(1) # $example on$ diff --git a/examples/kmeans-pyspark/run.sh b/examples/kmeans-pyspark/run.sh index 5647a84ae..4899cef2a 100755 --- a/examples/kmeans-pyspark/run.sh +++ b/examples/kmeans-pyspark/run.sh @@ -8,7 +8,7 @@ DATA_FILE=$HDFS_ROOT/data/sample_kmeans_data.txt APP_PY=kmeans-pyspark.py -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/kmeans/pom.xml b/examples/kmeans/pom.xml index 39a8ba99e..01c0016dd 100644 --- a/examples/kmeans/pom.xml +++ b/examples/kmeans/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar KMeansExample @@ -15,7 +15,7 @@ 1.2.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/examples/kmeans/run-gpu-standalone.sh b/examples/kmeans/run-gpu-standalone.sh index 411c1b4f4..5254e17b5 100755 --- a/examples/kmeans/run-gpu-standalone.sh +++ b/examples/kmeans/run-gpu-standalone.sh @@ -16,7 +16,7 @@ EXECUTOR_GPU_AMOUNT=1 TASK_GPU_AMOUNT=1 # Should run in standalone mode -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/kmeans/run.sh b/examples/kmeans/run.sh index 1d51e9444..8623d3134 100755 --- a/examples/kmeans/run.sh +++ b/examples/kmeans/run.sh @@ -9,7 +9,7 @@ DATA_FILE=$HDFS_ROOT/data/sample_kmeans_data.txt APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar APP_CLASS=org.apache.spark.examples.ml.KMeansExample -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/linear-regression/pom.xml b/examples/linear-regression/pom.xml index 98d0d7585..efa4e59db 100644 --- a/examples/linear-regression/pom.xml +++ b/examples/linear-regression/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar LinearRegressionExample @@ -15,7 +15,7 @@ 1.2.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/examples/linear-regression/run.sh b/examples/linear-regression/run.sh index b7a0b64b5..e25cb269d 100755 --- a/examples/linear-regression/run.sh +++ b/examples/linear-regression/run.sh @@ -9,7 +9,7 @@ DATA_FILE=$HDFS_ROOT/data/sample_linear_regression_data.txt APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar APP_CLASS=org.apache.spark.examples.ml.LinearRegressionExample -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/naive-bayes/pom.xml b/examples/naive-bayes/pom.xml index 9660c2968..13f91d75e 100644 --- a/examples/naive-bayes/pom.xml +++ b/examples/naive-bayes/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar NaiveBayesExample @@ -15,7 +15,7 @@ 1.2.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/examples/naive-bayes/run.sh b/examples/naive-bayes/run.sh index 2c2676429..e3f747c42 100755 --- a/examples/naive-bayes/run.sh +++ b/examples/naive-bayes/run.sh @@ -9,7 +9,7 @@ DATA_FILE=$HDFS_ROOT/data/sample_libsvm_data.txt APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar APP_CLASS=org.apache.spark.examples.ml.NaiveBayesExample -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/pca-pyspark/run.sh b/examples/pca-pyspark/run.sh index a9b5bd38c..dfdc343a1 100755 --- a/examples/pca-pyspark/run.sh +++ b/examples/pca-pyspark/run.sh @@ -9,7 +9,7 @@ DATA_FILE=$HDFS_ROOT/data/pca_data.csv APP_PY=pca-pyspark.py K=3 -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/pca/pom.xml b/examples/pca/pom.xml index 123b1654a..4db4ecc18 100644 --- a/examples/pca/pom.xml +++ b/examples/pca/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar PCAExample @@ -15,7 +15,7 @@ 1.2.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/examples/pca/run-gpu-standalone.sh b/examples/pca/run-gpu-standalone.sh index cfd0404ed..de699fc93 100755 --- a/examples/pca/run-gpu-standalone.sh +++ b/examples/pca/run-gpu-standalone.sh @@ -12,7 +12,7 @@ EXECUTOR_GPU_AMOUNT=1 TASK_GPU_AMOUNT=1 # Should run in standalone mode -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/pca/run.sh b/examples/pca/run.sh index 9ca29c6ee..749ff64fb 100755 --- a/examples/pca/run.sh +++ b/examples/pca/run.sh @@ -7,7 +7,7 @@ APP_CLASS=org.apache.spark.examples.ml.PCAExample # Dataset is created in the code, so no need to pass in as parameter -time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER \ --num-executors $SPARK_NUM_EXECUTORS \ --executor-cores $SPARK_EXECUTOR_CORES \ --total-executor-cores $SPARK_TOTAL_CORES \ diff --git a/examples/run-all-pyspark.sh b/examples/run-all-pyspark.sh index 2e153dbd8..dd316c9dd 100755 --- a/examples/run-all-pyspark.sh +++ b/examples/run-all-pyspark.sh @@ -5,6 +5,11 @@ exampleDirs=(kmeans-pyspark pca-pyspark als-pyspark) for dir in ${exampleDirs[*]} do cd $dir + echo + echo ========================== + echo Running $dir ... + echo ========================== + echo ./run.sh cd .. done diff --git a/examples/run-all-scala.sh b/examples/run-all-scala.sh index 00f9b2129..269341848 100755 --- a/examples/run-all-scala.sh +++ b/examples/run-all-scala.sh @@ -5,6 +5,11 @@ exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) for dir in ${exampleDirs[*]} do cd $dir + echo + echo ========================== + echo Running $dir ... + echo ========================== + echo ./run.sh cd .. done diff --git a/examples/summarizer/IntelGpuResourceFile.json b/examples/summarizer/IntelGpuResourceFile.json new file mode 100644 index 000000000..4b5c3cc98 --- /dev/null +++ b/examples/summarizer/IntelGpuResourceFile.json @@ -0,0 +1 @@ +[{"id":{"componentName": "spark.worker","resourceName":"gpu"},"addresses":["0","1","2","3"]}] diff --git a/examples/summarizer/build.sh b/examples/summarizer/build.sh new file mode 100644 index 000000000..da373645b --- /dev/null +++ b/examples/summarizer/build.sh @@ -0,0 +1,3 @@ +#!/usr/bin/env bash + +mvn clean package diff --git a/examples/summarizer/pom.xml b/examples/summarizer/pom.xml new file mode 100644 index 000000000..2793bc681 --- /dev/null +++ b/examples/summarizer/pom.xml @@ -0,0 +1,94 @@ + + 4.0.0 + + com.intel.oap + oap-mllib-examples + 1.2.0 + jar + + SummaryStatisticsExample + https://github.com/oap-project/oap-mllib.git + + + UTF-8 + 1.2.0 + 2.12.10 + 2.12 + 3.1.1 + + + + + + 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} + + + + + org.apache.spark + spark-mllib_2.12 + ${spark.version} + + + + + + + + + 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/summarizer/run-gpu-standalone.sh b/examples/summarizer/run-gpu-standalone.sh new file mode 100755 index 000000000..853b479db --- /dev/null +++ b/examples/summarizer/run-gpu-standalone.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + +# Data file is from Spark Examples (data/mllib/sample_kmeans_data.txt) and put in examples/data +# The data file should be copied to $HDFS_ROOT before running examples + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar +APP_CLASS=org.apache.spark.examples.ml.SummaryStatisticsExample + +USE_GPU=true +RESOURCE_FILE=$PWD/IntelGpuResourceFile.json +WORKER_GPU_AMOUNT=4 +EXECUTOR_GPU_AMOUNT=1 +TASK_GPU_AMOUNT=1 + +# Should run in standalone mode +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --total-executor-cores $SPARK_TOTAL_CORES \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --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.oap.mllib.useGPU=$USE_GPU" \ + --conf "spark.worker.resourcesFile=$RESOURCE_FILE" \ + --conf "spark.worker.resource.gpu.amount=$WORKER_GPU_AMOUNT" \ + --conf "spark.executor.resource.gpu.amount=$EXECUTOR_GPU_AMOUNT" \ + --conf "spark.task.resource.gpu.amount=$TASK_GPU_AMOUNT" \ + --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 \ + 2>&1 | tee Summarizer-$(date +%m%d_%H_%M_%S).log + diff --git a/examples/summarizer/run.sh b/examples/summarizer/run.sh new file mode 100644 index 000000000..6a8bab7d3 --- /dev/null +++ b/examples/summarizer/run.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar +APP_CLASS=org.apache.spark.examples.ml.SummaryStatisticsExample + +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --total-executor-cores $SPARK_TOTAL_CORES \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --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" \ + --conf "spark.eventLog.enabled=true" \ + --conf "spark.eventLog.dir=hdfs://beaver:9000/spark-history-server" \ + --conf "spark.history.fs.logDirectory=hdfs://beaver:9000/spark-history-server" \ + --conf "spark.driver.log.dfsDir=/user/spark/driverLogs" \ + --jars $OAP_MLLIB_JAR \ + --class $APP_CLASS \ + $APP_JAR \ + 2>&1 | tee Summarizer-$(date +%m%d_%H_%M_%S).log \ No newline at end of file diff --git a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala new file mode 100644 index 000000000..a173e2782 --- /dev/null +++ b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala @@ -0,0 +1,67 @@ +/* + * 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 + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector => NewVector} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.SparkSession +import org.apache.spark.rdd.RDD + +// $example off$ + +object SummaryStatisticsExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("SummaryStatisticsExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = sc.parallelize( Seq( + Vectors.dense(5.308206,9.869278,1.018934,4.292158,6.081011,6.585723,2.411094,4.767308,-3.256320,-6.029562), + Vectors.dense(7.279464,0.390664,-9.619284,3.435376,-4.769490,-4.873188,-0.118791,-5.117316,-0.418655,-0.475422), + Vectors.dense(-6.615791,-6.191542,0.402459,-9.743521,-9.990568,9.105346,1.691312,-2.605659,9.534952,-7.829027), + Vectors.dense(-4.792007,-2.491098,-2.939393,8.086467,3.773812,-9.997300,0.222378,8.995244,-5.753282,6.091060), + Vectors.dense(7.700725,-6.414918,1.684476,-8.983361,4.284580,-9.017608,0.552379,-7.705741,2.589852,0.411561), + Vectors.dense(6.991900,-1.063721,9.321163,-0.429719,-2.167696,-1.736425,-0.919139,6.980681,-0.711914,3.414347), + Vectors.dense(5.794488,-1.062261,0.955322,0.389642,3.012921,-9.953994,-3.197309,3.992421,-6.935902,8.147622), + Vectors.dense(-2.486670,6.973242,-4.047004,-5.655629,5.081786,5.533859,7.821403,2.763313,-0.454056,6.554309), + Vectors.dense(2.204855,7.839522,7.381886,1.618749,-6.566877,7.584285,-8.355983,-5.501410,-8.191205,-2.608499), + Vectors.dense(-9.948613,-8.941953,-8.106389,4.863542,5.852806,-1.659259,6.342504,-8.190106,-3.110330,-7.484658), + )) + + val summary: MultivariateStatisticalSummary = Statistics.colStats(data) + println(summary.mean) // a dense vector containing the mean value for each column + println(summary.variance) // column-wise variance + println(summary.max) + println(summary.min) + println(summary.count) + println(summary.normL1) + println(summary.normL2) + println(summary.weightSum) + println(summary.numNonzeros) // number of nonzeros in each column + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/mllib-dal/build-cpu-gpu.sh b/mllib-dal/build-cpu-gpu.sh deleted file mode 100755 index 4317471e1..000000000 --- a/mllib-dal/build-cpu-gpu.sh +++ /dev/null @@ -1,98 +0,0 @@ -#!/usr/bin/env bash - -# Check envs for building -if [[ -z $JAVA_HOME ]]; then - echo JAVA_HOME not defined! - exit 1 -fi - -if [[ -z $(which mvn) ]]; then - echo Maven not found! - exit 1 -fi - -if [[ -z $DAALROOT ]]; then - echo DAALROOT not defined! - exit 1 -fi - -if [[ -z $TBBROOT ]]; then - echo TBBROOT not defined! - exit 1 -fi - -if [[ -z $CCL_ROOT ]]; then - echo CCL_ROOT not defined! - exit 1 -fi - -# Check lib dependencies for building -RESOURCE_PATH=src/main/resources/lib -LIBS=(libccl.so.1 libfabric.so libfabric.so.1 libJavaAPI.so libmpi.so.12 \ - libsockets-fi.so libtbbmalloc.so.2 libtbb.so.12 libintlc.so.5 libsvml.so libirng.so libimf.so \ - libOpenCL.so.1 libsycl.so.5) -for lib in ${LIBS[@]} -do - if [[ ! -f ./$RESOURCE_PATH/$lib ]]; then - echo $RESOURCE_PATH/$lib does not exsit, please run ../dev/prepare-build-deps-gpu.sh! - exit 1 -fi -done - -versionArray=( - spark-3.0.0 \ - spark-3.0.1 \ - spark-3.0.2 \ - spark-3.1.1 -) - -SPARK_VER=spark-3.1.1 -MVN_NO_TRANSFER_PROGRESS= - -print_usage() { - echo - echo Usage: ./build.sh [-p spark-x.x.x] [-q] [-h] - echo - echo Supported Spark versions: - for version in ${versionArray[*]} - do - echo " $version" - done - echo -} - -while getopts "hqp:" opt -do -case $opt in - p) SPARK_VER=$OPTARG ;; - q) MVN_NO_TRANSFER_PROGRESS=--no-transfer-progress ;; - h | *) - print_usage - exit 1 - ;; -esac -done - -if [[ ! ${versionArray[*]} =~ $SPARK_VER ]]; then - echo Error: $SPARK_VER version is not supported! - exit 1 -fi - -export PLATFORM_PROFILE=CPU_GPU_PROFILE - -print_usage - -echo === Building Environments === -echo JAVA_HOME=$JAVA_HOME -echo DAALROOT=$DAALROOT -echo TBBROOT=$TBBROOT -echo CCL_ROOT=$CCL_ROOT -echo Maven Version: $(mvn -v | head -n 1 | cut -f3 -d" ") -echo Clang Version: $(clang -dumpversion) -echo Spark Version: $SPARK_VER -echo Platform Profile: $PLATFORM_PROFILE -echo ============================= -echo -echo Building with $SPARK_VER ... -echo -mvn $MVN_NO_TRANSFER_PROGRESS -P$SPARK_VER -DskipTests clean package diff --git a/mllib-dal/build.sh b/mllib-dal/build.sh index 96393f1ca..d8fece3b8 100755 --- a/mllib-dal/build.sh +++ b/mllib-dal/build.sh @@ -26,66 +26,102 @@ if [[ -z $CCL_ROOT ]]; then exit 1 fi -# Check lib dependencies for building -RESOURCE_PATH=src/main/resources/lib -LIBS=(libccl.so.1 libfabric.so libfabric.so.1 libJavaAPI.so libmpi.so.12 \ - libsockets-fi.so libtbbmalloc.so.2 libtbb.so.12) -for lib in ${LIBS[@]} -do - if [[ ! -f ./$RESOURCE_PATH/$lib ]]; then - echo $RESOURCE_PATH/$lib does not exsit, please run ../dev/prepare-build-deps.sh! - exit 1 -fi -done +check_cpu_libs() { + # Check lib dependencies for building + RESOURCE_PATH=src/main/resources/lib + LIBS=(libccl.so.1 libfabric.so libfabric.so.1 libJavaAPI.so libmpi.so.12 \ + libsockets-fi.so libtbbmalloc.so.2 libtbb.so.12) + for lib in ${LIBS[@]} + do + if [[ ! -f ./$RESOURCE_PATH/$lib ]]; then + echo \"$RESOURCE_PATH/$lib\" does not exsit, please run \"../dev/prepare-build-deps.sh\"! + exit 1 + fi + done -if [[ -f ./$RESOURCE_PATH/libsycl.so.5 ]]; then - echo GPU libs found! Please re-run ../dev/prepare-build-deps.sh! - exit 1 -fi + if [[ -f ./$RESOURCE_PATH/libsycl.so.5 ]]; then + echo + echo GPU libs found! Please re-run \"../dev/prepare-build-deps.sh\"! + echo + exit 1 + fi +} -versionArray=( - spark-3.0.0 \ - spark-3.0.1 \ - spark-3.0.2 \ - spark-3.1.1 -) +check_gpu_libs() { + # Check lib dependencies for building + RESOURCE_PATH=src/main/resources/lib + LIBS=(libccl.so.1 libfabric.so libfabric.so.1 libJavaAPI.so libmpi.so.12 \ + libsockets-fi.so libtbbmalloc.so.2 libtbb.so.12 libintlc.so.5 libsvml.so libirng.so libimf.so \ + libOpenCL.so.1 libsycl.so.5) + for lib in ${LIBS[@]} + do + if [[ ! -f ./$RESOURCE_PATH/$lib ]]; then + echo + echo \"$RESOURCE_PATH/$lib\" does not exsit, please run \"../dev/prepare-build-deps-gpu.sh\"! + echo + exit 1 + fi + done +} -SPARK_VER=spark-3.1.1 MVN_NO_TRANSFER_PROGRESS= print_usage() { echo - echo Usage: ./build.sh [-p spark-x.x.x] [-q] [-h] + echo "Usage: ./build.sh [-p ] [-q] [-h]" echo - echo Supported Spark versions: - for version in ${versionArray[*]} - do - echo " $version" - done + echo "-p Supported Platform Profiles:" + echo " CPU_ONLY_PROFILE" + echo " CPU_GPU_PROFILE" echo } -while getopts "hqp:" opt +while getopts "p:qh" opt do case $opt in - p) SPARK_VER=$OPTARG ;; + p) PLATFORM_OPT=$OPTARG ;; q) MVN_NO_TRANSFER_PROGRESS=--no-transfer-progress ;; h | *) - print_usage - exit 1 - ;; + print_usage + exit 1 + ;; esac done -if [[ ! ${versionArray[*]} =~ $SPARK_VER ]]; then - echo Error: $SPARK_VER version is not supported! +shift "$((OPTIND-1))" + +SUITE=$1 + +print_usage + +if [[ -n $PLATFORM_OPT && ! ($PLATFORM_OPT == CPU_ONLY_PROFILE || $PLATFORM_OPT == CPU_GPU_PROFILE) ]]; then + echo + echo Platform Profile should be CPU_ONLY_PROFILE or CPU_GPU_PROFILE, but \"$PLATFORM_OPT\" found! + echo exit 1 fi -print_usage +if [[ ! ${suiteArray[*]} =~ $SUITE ]]; then + echo Error: $SUITE test suite is not supported! + exit 1 +fi + +# Import RELEASE envs +SCRIPT_DIR=$( cd $(dirname ${BASH_SOURCE[0]}) && pwd ) +OAP_MLLIB_ROOT=$(cd $SCRIPT_DIR/.. && pwd) +source $OAP_MLLIB_ROOT/RELEASE -export PLATFORM_PROFILE=CPU_ONLY_PROFILE +export PLATFORM_PROFILE=${PLATFORM_OPT:-$PLATFORM_PROFILE} + +if [[ $PLATFORM_PROFILE == CPU_ONLY_PROFILE ]] +then + check_cpu_libs +elif [[ $PLATFORM_PROFILE == CPU_GPU_PROFILE ]] +then + check_gpu_libs +fi +echo echo === Building Environments === echo JAVA_HOME=$JAVA_HOME echo DAALROOT=$DAALROOT @@ -93,10 +129,9 @@ echo TBBROOT=$TBBROOT echo CCL_ROOT=$CCL_ROOT echo Maven Version: $(mvn -v | head -n 1 | cut -f3 -d" ") echo Clang Version: $(clang -dumpversion) -echo Spark Version: $SPARK_VER +echo Spark Version: $SPARK_VERSION echo Platform Profile: $PLATFORM_PROFILE echo ============================= echo -echo Building with $SPARK_VER ... -echo -mvn $MVN_NO_TRANSFER_PROGRESS -P$SPARK_VER -DskipTests clean package + +mvn $MVN_NO_TRANSFER_PROGRESS -DskipTests clean package diff --git a/mllib-dal/pom.xml b/mllib-dal/pom.xml index d2b74863d..5b14ed887 100644 --- a/mllib-dal/pom.xml +++ b/mllib-dal/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib - 1.2.0 + 1.3.0 OAP Project OAP MLlib jar https://github.com/oap-project/oap-mllib.git @@ -15,16 +15,9 @@ UTF-8 2.12.10 2.12 - 3.1.1 - 2021.3.0 - libtbb.so.12.3 - libtbbmalloc.so.2.3 - libJavaAPI.so.1.1 - libccl.so - libfabric.so.1 - libmpi.so.12.0.0 - libOpenCL.so.1 - libsycl.so.5 + 3.2.9 + 3.2.0 + 2021.4.0.83 src/assembly/assembly.xml @@ -92,7 +85,7 @@ com.intel.dal dal - 2021.4.0.83 + ${dal.version} junit @@ -162,83 +155,8 @@ - - - cpu-gpu - - - env.PLATFORM_PROFILE - CPU_GPU_PROFILE - - - - - spark-3.0.0 - - 3.0.0 - 3.0.8 - - - - spark-3.0.1 - - 3.0.1 - 3.0.8 - - - - spark-3.0.2 - - 3.0.2 - 3.0.8 - - - - spark-3.1.1 - - true - - - 3.1.1 - 3.2.3 - - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.2.0 - - - add-source - generate-sources - - add-source - - - - src/spark-${spark.version}/main/java - src/spark-${spark.version}/main/scala - - - - - add-test-source - generate-sources - - add-test-source - - - - src/spark-${spark.version}/test/scala - - - - - net.alchim31.maven scala-maven-plugin @@ -357,7 +275,7 @@ 1.8 - process-resources + generate-resources Building native code diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/util/CCLParam.java b/mllib-dal/src/main/java/com/intel/oap/mllib/CCLParam.java similarity index 95% rename from mllib-dal/src/main/java/org/apache/spark/ml/util/CCLParam.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/CCLParam.java index 38052d4cc..9118669fd 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/util/CCLParam.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/CCLParam.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.util; +package com.intel.oap.mllib; class CCLParam { long commSize; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java b/mllib-dal/src/main/java/com/intel/oap/mllib/LibLoader.java similarity index 98% rename from mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/LibLoader.java index 52a898efd..760cb1634 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/LibLoader.java @@ -14,9 +14,8 @@ * limitations under the License. */ -package org.apache.spark.ml.util; +package com.intel.oap.mllib; -import com.intel.daal.utils.LibUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/util/Service.java b/mllib-dal/src/main/java/com/intel/oap/mllib/Service.java similarity index 98% rename from mllib-dal/src/main/java/org/apache/spark/ml/util/Service.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/Service.java index 306cd5467..036b9b6af 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/util/Service.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/Service.java @@ -16,21 +16,16 @@ // Based on oneDAL Java example code -package org.apache.spark.ml.util; +package com.intel.oap.mllib; import com.intel.daal.data_management.data.CSRNumericTable; import com.intel.daal.data_management.data.HomogenNumericTable; import com.intel.daal.data_management.data.NumericTable; -import com.intel.daal.services.DaalContext; import com.intel.daal.services.ErrorHandling; -import java.io.BufferedReader; -import java.io.FileReader; -import java.io.IOException; import java.nio.FloatBuffer; import java.nio.IntBuffer; import java.text.DecimalFormat; -import java.util.ArrayList; public class Service { diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/classification/NaiveBayesResult.java similarity index 69% rename from mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/classification/NaiveBayesResult.java index bf5e51941..b44c4d941 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/classification/NaiveBayesResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/classification/NaiveBayesResult.java @@ -1,4 +1,4 @@ -package org.apache.spark.ml.classification; +package com.intel.oap.mllib.classification; public class NaiveBayesResult { public long piNumericTable; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/clustering/KMeansResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/clustering/KMeansResult.java similarity index 94% rename from mllib-dal/src/main/java/org/apache/spark/ml/clustering/KMeansResult.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/clustering/KMeansResult.java index eb7b0cd1e..9d07f55dc 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/clustering/KMeansResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/clustering/KMeansResult.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.clustering; +package com.intel.oap.mllib.clustering; public class KMeansResult { public int iterationNum; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/feature/PCAResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/feature/PCAResult.java similarity index 95% rename from mllib-dal/src/main/java/org/apache/spark/ml/feature/PCAResult.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/feature/PCAResult.java index 5d07c7e21..2f7b5b203 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/feature/PCAResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/feature/PCAResult.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.feature; +package com.intel.oap.mllib.feature; public class PCAResult { public long pcNumericTable; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java b/mllib-dal/src/main/java/com/intel/oap/mllib/recommendation/ALSPartitionInfo.java similarity index 93% rename from mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/recommendation/ALSPartitionInfo.java index 793350e79..accf11de6 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/recommendation/ALSPartitionInfo.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.recommendation; +package com.intel.oap.mllib.recommendation; public class ALSPartitionInfo { public int ratingsNum; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/recommendation/ALSResult.java similarity index 94% rename from mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/recommendation/ALSResult.java index c4824e896..1cadfb121 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/recommendation/ALSResult.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.recommendation; +package com.intel.oap.mllib.recommendation; public class ALSResult { public long rankId = -1; diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/regression/LiRResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/regression/LiRResult.java similarity index 95% rename from mllib-dal/src/main/java/org/apache/spark/ml/regression/LiRResult.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/regression/LiRResult.java index 65d30f85c..aa2311e85 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/regression/LiRResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/regression/LiRResult.java @@ -14,7 +14,7 @@ * limitations under the License. *******************************************************************************/ -package org.apache.spark.ml.regression; +package com.intel.oap.mllib.regression; public class LiRResult { public long coeffNumericTable; // first element of coeff is actually intercept diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/stat/CorrelationResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/stat/CorrelationResult.java similarity index 96% rename from mllib-dal/src/main/java/org/apache/spark/ml/stat/CorrelationResult.java rename to mllib-dal/src/main/java/com/intel/oap/mllib/stat/CorrelationResult.java index 047c042a7..18ef870e7 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/stat/CorrelationResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/stat/CorrelationResult.java @@ -14,7 +14,7 @@ * limitations under the License. *******************************************************************************/ -package org.apache.spark.ml.stat; +package com.intel.oap.mllib.stat; public class CorrelationResult { public long correlationNumericTable; diff --git a/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java b/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java new file mode 100644 index 000000000..d34485004 --- /dev/null +++ b/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java @@ -0,0 +1,9 @@ +package org.apache.spark.mllib.stat; + +public class SummarizerResult { + public long meanNumericTable; + public long varianceNumericTable; + public long minimumNumericTable; + public long maximumNumericTable; + +} \ No newline at end of file diff --git a/mllib-dal/src/main/native/ALSDALImpl.cpp b/mllib-dal/src/main/native/ALSDALImpl.cpp index 5f2de2544..d029d08dc 100644 --- a/mllib-dal/src/main/native/ALSDALImpl.cpp +++ b/mllib-dal/src/main/native/ALSDALImpl.cpp @@ -19,7 +19,7 @@ #include #include "OneCCL.h" -#include "org_apache_spark_ml_recommendation_ALSDALImpl.h" +#include "com_intel_oap_mllib_recommendation_ALSDALImpl.h" #include "service.h" #include "ALSShuffle.h" @@ -444,14 +444,8 @@ static size_t getOffsetFromOffsetTable(NumericTablePtr offsetTable) { return ret; } -/* - * Class: org_apache_spark_ml_recommendation_ALSDALImpl - * Method: cShuffleData - * Signature: - * (Ljava/nio/ByteBuffer;IILorg/apache/spark/ml/recommendation/ALSPartitionInfo;)Ljava/nio/ByteBuffer; - */ JNIEXPORT jobject JNICALL -Java_org_apache_spark_ml_recommendation_ALSDALImpl_cShuffleData( +Java_com_intel_oap_mllib_recommendation_ALSDALImpl_cShuffleData( JNIEnv *env, jobject obj, jobject dataBuffer, jint nTotalKeys, jint nBlocks, jobject infoObj) { // cout << "cShuffleData: rank " << rankId << endl; @@ -488,14 +482,8 @@ Java_org_apache_spark_ml_recommendation_ALSDALImpl_cShuffleData( return env->NewDirectByteBuffer(ratings, newRatingsNum * RATING_SIZE); } -/* - * Class: org_apache_spark_ml_recommendation_ALSDALImpl - * Method: cDALImplictALS - * Signature: (JJIIDDIIILorg/apache/spark/ml/recommendation/ALSResult;)J - */ - JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_recommendation_ALSDALImpl_cDALImplictALS( +Java_com_intel_oap_mllib_recommendation_ALSDALImpl_cDALImplictALS( JNIEnv *env, jobject obj, jlong numTableAddr, jlong nUsers, jint nFactors, jint maxIter, jdouble regParam, jdouble alpha, jint executor_num, jint executor_cores, jint partitionId, jobject resultObj) { diff --git a/mllib-dal/src/main/native/CorrelationDALImpl.cpp b/mllib-dal/src/main/native/CorrelationDALImpl.cpp index f2efb70ea..f81c5a72f 100644 --- a/mllib-dal/src/main/native/CorrelationDALImpl.cpp +++ b/mllib-dal/src/main/native/CorrelationDALImpl.cpp @@ -22,7 +22,7 @@ #endif #include "OneCCL.h" -#include "org_apache_spark_ml_stat_CorrelationDALImpl.h" +#include "com_intel_oap_mllib_stat_CorrelationDALImpl.h" #include "service.h" using namespace std; @@ -138,13 +138,8 @@ static void correlation_compute(JNIEnv *env, } } -/* - * Class: org_apache_spark_ml_stat_CorrelationDALImpl - * Method: cCorrelationTrainDAL - * Signature: (JJDDIILorg/apache/spark/ml/stat/CorrelationResult;)J - */ JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_stat_CorrelationDALImpl_cCorrelationTrainDAL( +Java_com_intel_oap_mllib_stat_CorrelationDALImpl_cCorrelationTrainDAL( JNIEnv *env, jobject obj, jlong pNumTabData, jint executor_num, jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, jobject resultObj) { diff --git a/mllib-dal/src/main/native/KMeansDALImpl.cpp b/mllib-dal/src/main/native/KMeansDALImpl.cpp index 47f1b5dae..0cf151e72 100644 --- a/mllib-dal/src/main/native/KMeansDALImpl.cpp +++ b/mllib-dal/src/main/native/KMeansDALImpl.cpp @@ -22,7 +22,7 @@ #endif #include "OneCCL.h" -#include "org_apache_spark_ml_clustering_KMeansDALImpl.h" +#include "com_intel_oap_mllib_clustering_KMeansDALImpl.h" #include "service.h" using namespace std; @@ -229,13 +229,8 @@ static jlong doKMeansDALComputeWithInitCenters( return (jlong)0; } -/* - * Class: org_apache_spark_ml_clustering_KMeansDALImpl - * Method: cKMeansDALComputeWithInitCenters - * Signature: (JJIDIIILorg/apache/spark/ml/clustering/KMeansResult;)J - */ JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_clustering_KMeansDALImpl_cKMeansDALComputeWithInitCenters( +Java_com_intel_oap_mllib_clustering_KMeansDALImpl_cKMeansDALComputeWithInitCenters( JNIEnv *env, jobject obj, jlong pNumTabData, jlong pNumTabCenters, jint cluster_num, jdouble tolerance, jint iteration_num, jint executor_num, jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, diff --git a/mllib-dal/src/main/native/LinearRegressionDALImpl.cpp b/mllib-dal/src/main/native/LinearRegressionDALImpl.cpp index 5755541d2..c061acfea 100644 --- a/mllib-dal/src/main/native/LinearRegressionDALImpl.cpp +++ b/mllib-dal/src/main/native/LinearRegressionDALImpl.cpp @@ -19,7 +19,7 @@ #include #include "OneCCL.h" -#include "org_apache_spark_ml_regression_LinearRegressionDALImpl.h" +#include "com_intel_oap_mllib_regression_LinearRegressionDALImpl.h" #include "service.h" using namespace std; @@ -194,13 +194,8 @@ static NumericTablePtr ridge_regression_compute( return resultTable; } -/* - * Class: org_apache_spark_ml_regression_LinearRegressionDALImpl - * Method: cLinearRegressionTrainDAL - * Signature: (JJDDIILorg/apache/spark/ml/regression/LiRResult;)J - */ JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_regression_LinearRegressionDALImpl_cLinearRegressionTrainDAL( +Java_com_intel_oap_mllib_regression_LinearRegressionDALImpl_cLinearRegressionTrainDAL( JNIEnv *env, jobject obj, jlong pNumTabData, jlong pNumTabLabel, jdouble regParam, jdouble elasticNetParam, jint executor_num, jint executor_cores, jobject resultObj) { diff --git a/mllib-dal/src/main/native/Makefile b/mllib-dal/src/main/native/Makefile index cdc79a071..fe170d3ad 100644 --- a/mllib-dal/src/main/native/Makefile +++ b/mllib-dal/src/main/native/Makefile @@ -59,7 +59,8 @@ CPP_SRCS += \ ./ALSDALImpl.cpp ./ALSShuffle.cpp \ ./NaiveBayesDALImpl.cpp \ ./LinearRegressionDALImpl.cpp \ - ./CorrelationDALImpl.cpp + ./CorrelationDALImpl.cpp \ + ./SummarizerDALImpl.cpp OBJS += \ ./OneCCL.o ./OneDAL.o ./service.o ./error_handling.o \ @@ -68,7 +69,8 @@ OBJS += \ ./ALSDALImpl.o ./ALSShuffle.o \ ./NaiveBayesDALImpl.o \ ./LinearRegressionDALImpl.o \ - ./CorrelationDALImpl.o + ./CorrelationDALImpl.o \ + ./SummarizerDALImpl.o DEFINES=-D$(PLATFORM_PROFILE) diff --git a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp index ad1119f2d..02ed6c504 100644 --- a/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp +++ b/mllib-dal/src/main/native/NaiveBayesDALImpl.cpp @@ -1,7 +1,7 @@ #include #include "OneCCL.h" -#include "org_apache_spark_ml_classification_NaiveBayesDALImpl.h" +#include "com_intel_oap_mllib_classification_NaiveBayesDALImpl.h" #include "service.h" #define PROFILE 1 @@ -121,13 +121,8 @@ trainModel(const ccl::communicator &comm, const NumericTablePtr &featuresTab, 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( +Java_com_intel_oap_mllib_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute( JNIEnv *env, jobject obj, jlong pFeaturesTab, jlong pLabelsTab, jint class_num, jint executor_num, jint executor_cores, jobject resultObj) { diff --git a/mllib-dal/src/main/native/OneCCL.cpp b/mllib-dal/src/main/native/OneCCL.cpp index 0565d8859..8f80f2487 100644 --- a/mllib-dal/src/main/native/OneCCL.cpp +++ b/mllib-dal/src/main/native/OneCCL.cpp @@ -29,7 +29,7 @@ #include #include "OneCCL.h" -#include "org_apache_spark_ml_util_OneCCL__.h" +#include "com_intel_oap_mllib_OneCCL__.h" extern const int ccl_root = 0; @@ -41,12 +41,7 @@ static std::vector g_comms; ccl::communicator &getComm() { return g_comms[0]; } -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: c_init - * Signature: (IILjava/lang/String;Lorg/apache/spark/ml/util/CCLParam;)I - */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1init( +JNIEXPORT jint JNICALL Java_com_intel_oap_mllib_OneCCL_00024_c_1init( JNIEnv *env, jobject obj, jint size, jint rank, jstring ip_port, jobject param) { @@ -87,12 +82,7 @@ JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1init( return 1; } -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: c_cleanup - * Signature: ()V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1cleanup( +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneCCL_00024_c_1cleanup( JNIEnv *env, jobject obj) { g_comms.pop_back(); @@ -100,33 +90,18 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1cleanup( std::cerr << "OneCCL (native): cleanup" << std::endl; } -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: isRoot - * Signature: ()Z - */ JNIEXPORT jboolean JNICALL -Java_org_apache_spark_ml_util_OneCCL_00024_isRoot(JNIEnv *env, jobject obj) { +Java_com_intel_oap_mllib_OneCCL_00024_isRoot(JNIEnv *env, jobject obj) { return getComm().rank() == 0; } -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: rankID - * Signature: ()I - */ JNIEXPORT jint JNICALL -Java_org_apache_spark_ml_util_OneCCL_00024_rankID(JNIEnv *env, jobject obj) { +Java_com_intel_oap_mllib_OneCCL_00024_rankID(JNIEnv *env, jobject obj) { return getComm().rank(); } -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: setEnv - * Signature: (Ljava/lang/String;Ljava/lang/String;Z)I - */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_setEnv( +JNIEXPORT jint JNICALL Java_com_intel_oap_mllib_OneCCL_00024_setEnv( JNIEnv *env, jobject obj, jstring key, jstring value, jboolean overwrite) { char *k = (char *)env->GetStringUTFChars(key, NULL); @@ -201,13 +176,8 @@ static bool is_valid_ip(char ip[]) { return false; } -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: getAvailPort - * Signature: (Ljava/lang/String;)I - */ JNIEXPORT jint JNICALL -Java_org_apache_spark_ml_util_OneCCL_00024_c_1getAvailPort(JNIEnv *env, +Java_com_intel_oap_mllib_OneCCL_00024_c_1getAvailPort(JNIEnv *env, jobject obj, jstring localIP) { diff --git a/mllib-dal/src/main/native/OneDAL.cpp b/mllib-dal/src/main/native/OneDAL.cpp index 8aadda4f0..af6c2bd14 100644 --- a/mllib-dal/src/main/native/OneDAL.cpp +++ b/mllib-dal/src/main/native/OneDAL.cpp @@ -17,7 +17,7 @@ #include #include -#include "org_apache_spark_ml_util_OneDAL__.h" +#include "com_intel_oap_mllib_OneDAL__.h" #include "service.h" using namespace daal; @@ -26,12 +26,18 @@ using namespace daal::data_management; // Use oneDAL lib function extern bool daal_check_is_intel_cpu(); -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cSetDouble - * Signature: (JIID)V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDouble( +JNIEXPORT void JNICALL +Java_com_intel_oap_mllib_OneDAL_00024_cAddNumericTable( + JNIEnv *, jobject, jlong rowMergedNumericTableAddr, + jlong numericTableAddr) { + data_management::RowMergedNumericTablePtr pRowMergedNumericTable = (*( + data_management::RowMergedNumericTablePtr *)rowMergedNumericTableAddr); + data_management::NumericTablePtr pNumericTable = + (*(data_management::NumericTablePtr *)numericTableAddr); + pRowMergedNumericTable->addNumericTable(pNumericTable); +} + +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cSetDouble( JNIEnv *env, jobject, jlong numTableAddr, jint row, jint column, jdouble value) { HomogenNumericTable *nt = @@ -46,7 +52,7 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDouble( * Signature: (JI[DII)V */ JNIEXPORT void JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_cSetDoubleBatch( +Java_com_intel_oap_mllib_OneDAL_00024_cSetDoubleBatch( JNIEnv *env, jobject, jlong numTableAddr, jint curRows, jdoubleArray batch, jint numRows, jint numCols) { HomogenNumericTable *nt = @@ -57,55 +63,24 @@ Java_org_apache_spark_ml_util_OneDAL_00024_cSetDoubleBatch( env->ReleasePrimitiveArrayCritical(batch, values, JNI_ABORT); } -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cAddNumericTable - * Signature: (JJ)V - */ -JNIEXPORT void JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_cAddNumericTable( - JNIEnv *, jobject, jlong rowMergedNumericTableAddr, - jlong numericTableAddr) { - data_management::RowMergedNumericTablePtr pRowMergedNumericTable = (*( - data_management::RowMergedNumericTablePtr *)rowMergedNumericTableAddr); - data_management::NumericTablePtr pNumericTable = - (*(data_management::NumericTablePtr *)numericTableAddr); - pRowMergedNumericTable->addNumericTable(pNumericTable); -} - -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cFreeDataMemory - * Signature: (J)V - */ JNIEXPORT void JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_cFreeDataMemory( +Java_com_intel_oap_mllib_OneDAL_00024_cFreeDataMemory( JNIEnv *, jobject, jlong numericTableAddr) { data_management::NumericTablePtr pNumericTable = (*(data_management::NumericTablePtr *)numericTableAddr); pNumericTable->freeDataMemory(); } -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cCheckPlatformCompatibility - * Signature: ()Z - */ JNIEXPORT jboolean JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_cCheckPlatformCompatibility( +Java_com_intel_oap_mllib_OneDAL_00024_cCheckPlatformCompatibility( JNIEnv *, jobject) { // Only guarantee compatibility and performance on Intel platforms, use // oneDAL lib function return daal_check_is_intel_cpu(); } -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cNewCSRNumericTableFloat - * Signature: ([F[J[JJJ)J - */ JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableFloat( +Java_com_intel_oap_mllib_OneDAL_00024_cNewCSRNumericTableFloat( JNIEnv *env, jobject, jfloatArray data, jlongArray colIndices, jlongArray rowOffsets, jlong nFeatures, jlong nVectors) { @@ -142,13 +117,8 @@ Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTableFloat( 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( +Java_com_intel_oap_mllib_OneDAL_00024_cNewCSRNumericTableDouble( JNIEnv *env, jobject, jdoubleArray data, jlongArray colIndices, jlongArray rowOffsets, jlong nFeatures, jlong nVectors) { diff --git a/mllib-dal/src/main/native/PCADALImpl.cpp b/mllib-dal/src/main/native/PCADALImpl.cpp index 0f34fa87b..f84f3a122 100644 --- a/mllib-dal/src/main/native/PCADALImpl.cpp +++ b/mllib-dal/src/main/native/PCADALImpl.cpp @@ -22,7 +22,7 @@ #endif #include "OneCCL.h" -#include "org_apache_spark_ml_feature_PCADALImpl.h" +#include "com_intel_oap_mllib_feature_PCADALImpl.h" #include "service.h" using namespace std; @@ -162,14 +162,8 @@ static void doPCADALCompute(JNIEnv *env, jobject obj, int rankId, } } -/* - * Class: org_apache_spark_ml_feature_PCADALImpl - * Method: cPCATrainDAL - * Signature: (JIIIZ[ILorg/apache/spark/ml/feature/PCAResult;)J - */ - JNIEXPORT jlong JNICALL -Java_org_apache_spark_ml_feature_PCADALImpl_cPCATrainDAL( +Java_com_intel_oap_mllib_feature_PCADALImpl_cPCATrainDAL( JNIEnv *env, jobject obj, jlong pNumTabData, jint k, jint executor_num, jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, jobject resultObj) { diff --git a/mllib-dal/src/main/native/SummarizerDALImpl.cpp b/mllib-dal/src/main/native/SummarizerDALImpl.cpp new file mode 100644 index 000000000..6428c2696 --- /dev/null +++ b/mllib-dal/src/main/native/SummarizerDALImpl.cpp @@ -0,0 +1,250 @@ +/******************************************************************************* + * 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. + *******************************************************************************/ + +#include +#include + +#ifdef CPU_GPU_PROFILE +#include "GPU.h" +#endif + +#include "OneCCL.h" +#include "org_apache_spark_mllib_stat_SummarizerDALImpl.h" +#include "service.h" + +using namespace std; +using namespace daal; +using namespace daal::algorithms; + +typedef double algorithmFPType; /* Algorithm floating-point type */ + +static void summarizer_compute(JNIEnv *env, + jobject obj, + int rankId, + ccl::communicator &comm, + const NumericTablePtr &pData, + size_t nBlocks, + jobject resultObj) { + using daal::byte; + auto t1 = std::chrono::high_resolution_clock::now(); + + const bool isRoot = (rankId == ccl_root); + + low_order_moments::Distributed localAlgorithm; + + /* Set the input data set to the algorithm */ + localAlgorithm.input.set(low_order_moments::data, pData); + + /* Compute low_order_moments */ + localAlgorithm.compute(); + + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): local step took " << duration << " secs" + << std::endl; + + t1 = std::chrono::high_resolution_clock::now(); + + /* Serialize partial results required by step 2 */ + services::SharedPtr serializedData; + InputDataArchive dataArch; + localAlgorithm.getPartialResult()->serialize(dataArch); + size_t perNodeArchLength = dataArch.getSizeOfArchive(); + + serializedData = + services::SharedPtr(new byte[perNodeArchLength * nBlocks]); + + byte *nodeResults = new byte[perNodeArchLength]; + dataArch.copyArchiveToArray(nodeResults, perNodeArchLength); + std::vector aReceiveCount(comm.size(), + perNodeArchLength); // 4 x "14016" + + /* Transfer partial results to step 2 on the root node */ + ccl::gather((int8_t *)nodeResults, perNodeArchLength, + (int8_t *)(serializedData.get()), perNodeArchLength, comm) + .wait(); + t2 = std::chrono::high_resolution_clock::now(); + + duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): ccl_gather took " << duration << " secs" + << std::endl; + if (isRoot) { + auto t1 = std::chrono::high_resolution_clock::now(); + /* Create an algorithm to compute covariance on the master node */ + low_order_moments::Distributed masterAlgorithm; + + for (size_t i = 0; i < nBlocks; i++) { + /* Deserialize partial results from step 1 */ + OutputDataArchive dataArch(serializedData.get() + + perNodeArchLength * i, + perNodeArchLength); + + low_order_moments::PartialResultPtr dataForStep2FromStep1(new low_order_moments::PartialResult()); + dataForStep2FromStep1->deserialize(dataArch); + + /* Set local partial results as input for the master-node algorithm + */ + masterAlgorithm.input.add(low_order_moments::partialResults, + dataForStep2FromStep1); + } + + /* Set the parameter to choose the type of the output matrix */ + masterAlgorithm.parameter.estimatesToCompute = low_order_moments::estimatesAll; + + /* Merge and finalizeCompute covariance decomposition on the master node */ + masterAlgorithm.compute(); + masterAlgorithm.finalizeCompute(); + + /* Retrieve the algorithm results */ + low_order_moments::ResultPtr result = masterAlgorithm.getResult(); + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): master step took " << duration << " secs" + << std::endl; + + /* Print the results */ + printNumericTable(result->get(low_order_moments::mean), + "low_order_moments first 20 columns of " + "Mean :", + 1, 20); + printNumericTable(result->get(low_order_moments::variance), + "low_order_moments first 20 columns of " + "Variance :", + 1, 20); + printNumericTable(result->get(low_order_moments::minimum), + "low_order_moments first 20 columns of " + "Minimum :", + 1, 20); + printNumericTable(result->get(low_order_moments::maximum), + "low_order_moments first 20 columns of " + "Maximum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sum), + "low_order_moments first 20 columns of " + "Sum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquares), + "low_order_moments first 20 columns of " + "SumSquares :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquaresCentered), + "low_order_moments first 20 columns of " + "SumSquaresCentered :", + 1, 20); + printNumericTable(result->get(low_order_moments::secondOrderRawMoment), + "low_order_moments first 20 columns of " + "SecondOrderRawMoment :", + 1, 20); + printNumericTable(result->get(low_order_moments::standardDeviation), + "low_order_moments first 20 columns of " + "StandardDeviation :", + 1, 20); + printNumericTable(result->get(low_order_moments::variation), + "low_order_moments first 20 columns of " + "Variation :", + 1, 20); + // Return all covariance & mean + jclass clazz = env->GetObjectClass(resultObj); + + // Get Field references + jfieldID meanNumericTableField = + env->GetFieldID(clazz, "meanNumericTable", "J"); + jfieldID varianceNumericTableField = + env->GetFieldID(clazz, "varianceNumericTable", "J"); + jfieldID minimumNumericTableField = + env->GetFieldID(clazz, "minimumNumericTable", "J"); + jfieldID maximumNumericTableField = + env->GetFieldID(clazz, "maximumNumericTable", "J"); + + NumericTablePtr *mean = + new NumericTablePtr(result->get(low_order_moments::mean)); + NumericTablePtr *variance = + new NumericTablePtr(result->get(low_order_moments::variance)); + NumericTablePtr *max = + new NumericTablePtr(result->get(low_order_moments::maximum)); + NumericTablePtr *min = + new NumericTablePtr(result->get(low_order_moments::minimum)); + + env->SetLongField(resultObj, meanNumericTableField, (jlong)mean); + env->SetLongField(resultObj, varianceNumericTableField, (jlong)variance); + env->SetLongField(resultObj, maximumNumericTableField, (jlong)max); + env->SetLongField(resultObj, minimumNumericTableField, (jlong)min); + + } +} + +/* + * Class: org_apache_spark_mllib_stat_CorrelationDALImpl + * Method: cCorrelationTrainDAL + * Signature: (JJDDIILorg/apache/spark/ml/stat/CorrelationResult;)J + */ +JNIEXPORT jlong JNICALL +Java_org_apache_spark_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL( + JNIEnv *env, jobject obj, jlong pNumTabData, + jint executor_num, jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, jobject resultObj) { + + ccl::communicator &comm = getComm(); + size_t rankId = comm.rank(); + std::cout << " rankId : " << rankId << " ! " + << std::endl; + + const size_t nBlocks = executor_num; + + NumericTablePtr pData = *((NumericTablePtr *)pNumTabData); + + #ifdef CPU_GPU_PROFILE + + if (use_gpu) { + int n_gpu = env->GetArrayLength(gpu_idx_array); + cout << "oneDAL (native): use GPU kernels with " << n_gpu << " GPU(s)" + << endl; + + jint *gpu_indices = env->GetIntArrayElements(gpu_idx_array, 0); + + int size = comm.size(); + auto assigned_gpu = + getAssignedGPU(comm, size, rankId, gpu_indices, n_gpu); + + // Set SYCL context + cl::sycl::queue queue(assigned_gpu); + daal::services::SyclExecutionContext ctx(queue); + daal::services::Environment::getInstance()->setDefaultExecutionContext( + ctx); + + summarizer_compute( + env, obj, rankId, comm, pData, nBlocks, resultObj); + + env->ReleaseIntArrayElements(gpu_idx_array, gpu_indices, 0); + } else + #endif + { + // 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; + + summarizer_compute( + env, obj, rankId, comm, pData, nBlocks, resultObj); + } + + return 0; +} diff --git a/mllib-dal/src/main/native/build-jni.sh b/mllib-dal/src/main/native/build-jni.sh index c58385d40..f1a265b8e 100755 --- a/mllib-dal/src/main/native/build-jni.sh +++ b/mllib-dal/src/main/native/build-jni.sh @@ -29,6 +29,16 @@ if [[ ! -e "$SPARK_HOME" ]]; then fi javah -d $WORK_DIR/javah -classpath "$WORK_DIR/../../../target/classes:$DAAL_JAR:$SPARK_HOME/jars/*" -force \ +<<<<<<< HEAD + com.intel.oap.mllib.OneCCL$ \ + com.intel.oap.mllib.OneDAL$ \ + com.intel.oap.mllib.clustering.KMeansDALImpl \ + com.intel.oap.mllib.feature.PCADALImpl \ + com.intel.oap.mllib.recommendation.ALSDALImpl \ + com.intel.oap.mllib.classification.NaiveBayesDALImpl \ + com.intel.oap.mllib.regression.LinearRegressionDALImpl \ + com.intel.oap.mllib.stat.CorrelationDALImpl +======= org.apache.spark.ml.util.OneCCL$ \ org.apache.spark.ml.util.OneDAL$ \ org.apache.spark.ml.clustering.KMeansDALImpl \ @@ -36,4 +46,6 @@ javah -d $WORK_DIR/javah -classpath "$WORK_DIR/../../../target/classes:$DAAL_JAR org.apache.spark.ml.recommendation.ALSDALImpl \ org.apache.spark.ml.classification.NaiveBayesDALImpl \ org.apache.spark.ml.regression.LinearRegressionDALImpl \ - org.apache.spark.ml.stat.CorrelationDALImpl + org.apache.spark.ml.stat.CorrelationDALImpl \ + org.apache.spark.mllib.stat.SummarizerDALImpl +>>>>>>> 1. enable Summarizer with one API diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneCCL__.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneCCL__.h new file mode 100644 index 000000000..ffa85424e --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneCCL__.h @@ -0,0 +1,61 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_OneCCL__ */ + +#ifndef _Included_com_intel_oap_mllib_OneCCL__ +#define _Included_com_intel_oap_mllib_OneCCL__ +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_OneCCL__ + * Method: c_init + * Signature: (IILjava/lang/String;Lcom/intel/oap/mllib/CCLParam;)I + */ +JNIEXPORT jint JNICALL Java_com_intel_oap_mllib_OneCCL_00024_c_1init + (JNIEnv *, jobject, jint, jint, jstring, jobject); + +/* + * Class: com_intel_oap_mllib_OneCCL__ + * Method: c_cleanup + * Signature: ()V + */ +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneCCL_00024_c_1cleanup + (JNIEnv *, jobject); + +/* + * Class: com_intel_oap_mllib_OneCCL__ + * Method: isRoot + * Signature: ()Z + */ +JNIEXPORT jboolean JNICALL Java_com_intel_oap_mllib_OneCCL_00024_isRoot + (JNIEnv *, jobject); + +/* + * Class: com_intel_oap_mllib_OneCCL__ + * Method: rankID + * Signature: ()I + */ +JNIEXPORT jint JNICALL Java_com_intel_oap_mllib_OneCCL_00024_rankID + (JNIEnv *, jobject); + +/* + * Class: com_intel_oap_mllib_OneCCL__ + * Method: setEnv + * Signature: (Ljava/lang/String;Ljava/lang/String;Z)I + */ +JNIEXPORT jint JNICALL Java_com_intel_oap_mllib_OneCCL_00024_setEnv + (JNIEnv *, jobject, jstring, jstring, jboolean); + +/* + * Class: com_intel_oap_mllib_OneCCL__ + * Method: c_getAvailPort + * Signature: (Ljava/lang/String;)I + */ +JNIEXPORT jint JNICALL Java_com_intel_oap_mllib_OneCCL_00024_c_1getAvailPort + (JNIEnv *, jobject, jstring); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneDAL__.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneDAL__.h new file mode 100644 index 000000000..bf1a4388d --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_OneDAL__.h @@ -0,0 +1,69 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_OneDAL__ */ + +#ifndef _Included_com_intel_oap_mllib_OneDAL__ +#define _Included_com_intel_oap_mllib_OneDAL__ +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cAddNumericTable + * Signature: (JJ)V + */ +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cAddNumericTable + (JNIEnv *, jobject, jlong, jlong); + +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cSetDouble + * Signature: (JIID)V + */ +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cSetDouble + (JNIEnv *, jobject, jlong, jint, jint, jdouble); + +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cSetDoubleBatch + * Signature: (JI[DII)V + */ +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cSetDoubleBatch + (JNIEnv *, jobject, jlong, jint, jdoubleArray, jint, jint); + +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cFreeDataMemory + * Signature: (J)V + */ +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cFreeDataMemory + (JNIEnv *, jobject, jlong); + +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cCheckPlatformCompatibility + * Signature: ()Z + */ +JNIEXPORT jboolean JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cCheckPlatformCompatibility + (JNIEnv *, jobject); + +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cNewCSRNumericTableFloat + * Signature: ([F[J[JJJ)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cNewCSRNumericTableFloat + (JNIEnv *, jobject, jfloatArray, jlongArray, jlongArray, jlong, jlong); + +/* + * Class: com_intel_oap_mllib_OneDAL__ + * Method: cNewCSRNumericTableDouble + * Signature: ([D[J[JJJ)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_OneDAL_00024_cNewCSRNumericTableDouble + (JNIEnv *, jobject, jdoubleArray, jlongArray, jlongArray, jlong, jlong); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_classification_NaiveBayesDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_classification_NaiveBayesDALImpl.h new file mode 100644 index 000000000..f3b203044 --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_classification_NaiveBayesDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_classification_NaiveBayesDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_classification_NaiveBayesDALImpl +#define _Included_com_intel_oap_mllib_classification_NaiveBayesDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_classification_NaiveBayesDALImpl + * Method: cNaiveBayesDALCompute + * Signature: (JJIIILcom/intel/oap/mllib/classification/NaiveBayesResult;)V + */ +JNIEXPORT void JNICALL Java_com_intel_oap_mllib_classification_NaiveBayesDALImpl_cNaiveBayesDALCompute + (JNIEnv *, jobject, jlong, jlong, jint, jint, jint, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_clustering_KMeansDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_clustering_KMeansDALImpl.h new file mode 100644 index 000000000..4753ea80e --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_clustering_KMeansDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_clustering_KMeansDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_clustering_KMeansDALImpl +#define _Included_com_intel_oap_mllib_clustering_KMeansDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_clustering_KMeansDALImpl + * Method: cKMeansDALComputeWithInitCenters + * Signature: (JJIDIIIZ[ILcom/intel/oap/mllib/clustering/KMeansResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_clustering_KMeansDALImpl_cKMeansDALComputeWithInitCenters + (JNIEnv *, jobject, jlong, jlong, jint, jdouble, jint, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_feature_PCADALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_feature_PCADALImpl.h new file mode 100644 index 000000000..3756bc2a3 --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_feature_PCADALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_feature_PCADALImpl */ + +#ifndef _Included_com_intel_oap_mllib_feature_PCADALImpl +#define _Included_com_intel_oap_mllib_feature_PCADALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_feature_PCADALImpl + * Method: cPCATrainDAL + * Signature: (JIIIZ[ILcom/intel/oap/mllib/feature/PCAResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_feature_PCADALImpl_cPCATrainDAL + (JNIEnv *, jobject, jlong, jint, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_recommendation_ALSDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_recommendation_ALSDALImpl.h new file mode 100644 index 000000000..f54e634b9 --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_recommendation_ALSDALImpl.h @@ -0,0 +1,29 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_recommendation_ALSDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_recommendation_ALSDALImpl +#define _Included_com_intel_oap_mllib_recommendation_ALSDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_recommendation_ALSDALImpl + * Method: cDALImplictALS + * Signature: (JJIIDDIIILcom/intel/oap/mllib/recommendation/ALSResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_recommendation_ALSDALImpl_cDALImplictALS + (JNIEnv *, jobject, jlong, jlong, jint, jint, jdouble, jdouble, jint, jint, jint, jobject); + +/* + * Class: com_intel_oap_mllib_recommendation_ALSDALImpl + * Method: cShuffleData + * Signature: (Ljava/nio/ByteBuffer;IILcom/intel/oap/mllib/recommendation/ALSPartitionInfo;)Ljava/nio/ByteBuffer; + */ +JNIEXPORT jobject JNICALL Java_com_intel_oap_mllib_recommendation_ALSDALImpl_cShuffleData + (JNIEnv *, jobject, jobject, jint, jint, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_regression_LinearRegressionDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_regression_LinearRegressionDALImpl.h new file mode 100644 index 000000000..ae05d382f --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_regression_LinearRegressionDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_regression_LinearRegressionDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_regression_LinearRegressionDALImpl +#define _Included_com_intel_oap_mllib_regression_LinearRegressionDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_regression_LinearRegressionDALImpl + * Method: cLinearRegressionTrainDAL + * Signature: (JJDDIILcom/intel/oap/mllib/regression/LiRResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_regression_LinearRegressionDALImpl_cLinearRegressionTrainDAL + (JNIEnv *, jobject, jlong, jlong, jdouble, jdouble, jint, jint, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_CorrelationDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_CorrelationDALImpl.h new file mode 100644 index 000000000..0be43ba84 --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_CorrelationDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_stat_CorrelationDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_stat_CorrelationDALImpl +#define _Included_com_intel_oap_mllib_stat_CorrelationDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_stat_CorrelationDALImpl + * Method: cCorrelationTrainDAL + * Signature: (JIIZ[ILcom/intel/oap/mllib/stat/CorrelationResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_stat_CorrelationDALImpl_cCorrelationTrainDAL + (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif 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 deleted file mode 100644 index 5c55d2e5b..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_classification_NaiveBayesDALImpl.h +++ /dev/null @@ -1,21 +0,0 @@ -/* 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/native/javah/org_apache_spark_ml_clustering_KMeansDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_clustering_KMeansDALImpl.h deleted file mode 100644 index 5d8405618..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_clustering_KMeansDALImpl.h +++ /dev/null @@ -1,21 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_clustering_KMeansDALImpl */ - -#ifndef _Included_org_apache_spark_ml_clustering_KMeansDALImpl -#define _Included_org_apache_spark_ml_clustering_KMeansDALImpl -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_clustering_KMeansDALImpl - * Method: cKMeansDALComputeWithInitCenters - * Signature: (JJIDIIIZ[ILorg/apache/spark/ml/clustering/KMeansResult;)J - */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_clustering_KMeansDALImpl_cKMeansDALComputeWithInitCenters - (JNIEnv *, jobject, jlong, jlong, jint, jdouble, jint, jint, jint, jboolean, jintArray, jobject); - -#ifdef __cplusplus -} -#endif -#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_feature_PCADALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_feature_PCADALImpl.h deleted file mode 100644 index 77bf1fe75..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_feature_PCADALImpl.h +++ /dev/null @@ -1,21 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_feature_PCADALImpl */ - -#ifndef _Included_org_apache_spark_ml_feature_PCADALImpl -#define _Included_org_apache_spark_ml_feature_PCADALImpl -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_feature_PCADALImpl - * Method: cPCATrainDAL - * Signature: (JIIIZ[ILorg/apache/spark/ml/feature/PCAResult;)J - */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_feature_PCADALImpl_cPCATrainDAL - (JNIEnv *, jobject, jlong, jint, jint, jint, jboolean, jintArray, jobject); - -#ifdef __cplusplus -} -#endif -#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_recommendation_ALSDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_recommendation_ALSDALImpl.h deleted file mode 100644 index 73024d05b..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_recommendation_ALSDALImpl.h +++ /dev/null @@ -1,29 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_recommendation_ALSDALImpl */ - -#ifndef _Included_org_apache_spark_ml_recommendation_ALSDALImpl -#define _Included_org_apache_spark_ml_recommendation_ALSDALImpl -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_recommendation_ALSDALImpl - * Method: cDALImplictALS - * Signature: (JJIIDDIIILorg/apache/spark/ml/recommendation/ALSResult;)J - */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_recommendation_ALSDALImpl_cDALImplictALS - (JNIEnv *, jobject, jlong, jlong, jint, jint, jdouble, jdouble, jint, jint, jint, jobject); - -/* - * Class: org_apache_spark_ml_recommendation_ALSDALImpl - * Method: cShuffleData - * Signature: (Ljava/nio/ByteBuffer;IILorg/apache/spark/ml/recommendation/ALSPartitionInfo;)Ljava/nio/ByteBuffer; - */ -JNIEXPORT jobject JNICALL Java_org_apache_spark_ml_recommendation_ALSDALImpl_cShuffleData - (JNIEnv *, jobject, jobject, jint, jint, jobject); - -#ifdef __cplusplus -} -#endif -#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_regression_LinearRegressionDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_regression_LinearRegressionDALImpl.h deleted file mode 100644 index bd8d85800..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_regression_LinearRegressionDALImpl.h +++ /dev/null @@ -1,21 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_regression_LinearRegressionDALImpl */ - -#ifndef _Included_org_apache_spark_ml_regression_LinearRegressionDALImpl -#define _Included_org_apache_spark_ml_regression_LinearRegressionDALImpl -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_regression_LinearRegressionDALImpl - * Method: cLinearRegressionTrainDAL - * Signature: (JJDDIILorg/apache/spark/ml/regression/LiRResult;)J - */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_regression_LinearRegressionDALImpl_cLinearRegressionTrainDAL - (JNIEnv *, jobject, jlong, jlong, jdouble, jdouble, jint, jint, jobject); - -#ifdef __cplusplus -} -#endif -#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_stat_CorrelationDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_stat_CorrelationDALImpl.h deleted file mode 100644 index 4cc53b475..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_stat_CorrelationDALImpl.h +++ /dev/null @@ -1,21 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_stat_CorrelationDALImpl */ - -#ifndef _Included_org_apache_spark_ml_stat_CorrelationDALImpl -#define _Included_org_apache_spark_ml_stat_CorrelationDALImpl -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_stat_CorrelationDALImpl - * Method: cCorrelationTrainDAL - * Signature: (JIIZ[ILorg/apache/spark/ml/stat/CorrelationResult;)J - */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_stat_CorrelationDALImpl_cCorrelationTrainDAL - (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); - -#ifdef __cplusplus -} -#endif -#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneCCL__.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneCCL__.h deleted file mode 100644 index 580c34bf9..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneCCL__.h +++ /dev/null @@ -1,61 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_util_OneCCL__ */ - -#ifndef _Included_org_apache_spark_ml_util_OneCCL__ -#define _Included_org_apache_spark_ml_util_OneCCL__ -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: c_init - * Signature: (IILjava/lang/String;Lorg/apache/spark/ml/util/CCLParam;)I - */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1init - (JNIEnv *, jobject, jint, jint, jstring, jobject); - -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: c_cleanup - * Signature: ()V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1cleanup - (JNIEnv *, jobject); - -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: isRoot - * Signature: ()Z - */ -JNIEXPORT jboolean JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_isRoot - (JNIEnv *, jobject); - -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: rankID - * Signature: ()I - */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_rankID - (JNIEnv *, jobject); - -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: setEnv - * Signature: (Ljava/lang/String;Ljava/lang/String;Z)I - */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_setEnv - (JNIEnv *, jobject, jstring, jstring, jboolean); - -/* - * Class: org_apache_spark_ml_util_OneCCL__ - * Method: c_getAvailPort - * Signature: (Ljava/lang/String;)I - */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1getAvailPort - (JNIEnv *, jobject, jstring); - -#ifdef __cplusplus -} -#endif -#endif 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 deleted file mode 100644 index fa98ae25e..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h +++ /dev/null @@ -1,69 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_ml_util_OneDAL__ */ - -#ifndef _Included_org_apache_spark_ml_util_OneDAL__ -#define _Included_org_apache_spark_ml_util_OneDAL__ -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cAddNumericTable - * Signature: (JJ)V - */ -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 - * Signature: (JI[DII)V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cSetDoubleBatch - (JNIEnv *, jobject, jlong, jint, jdoubleArray, jint, jint); - -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cFreeDataMemory - * Signature: (J)V - */ -JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cFreeDataMemory - (JNIEnv *, jobject, jlong); - -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cCheckPlatformCompatibility - * Signature: ()Z - */ -JNIEXPORT jboolean JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cCheckPlatformCompatibility - (JNIEnv *, jobject); - -/* - * Class: org_apache_spark_ml_util_OneDAL__ - * Method: cNewCSRNumericTableFloat - * Signature: ([F[J[JJJ)J - */ -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 -#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h new file mode 100644 index 000000000..4f0681069 --- /dev/null +++ b/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class org_apache_spark_mllib_stat_SummarizerDALImpl */ + +#ifndef _Included_org_apache_spark_mllib_stat_SummarizerDALImpl +#define _Included_org_apache_spark_mllib_stat_SummarizerDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: org_apache_spark_mllib_stat_SummarizerDALImpl + * Method: cSummarizerTrainDAL + * Signature: (JIIZ[ILorg/apache/spark/mllib/stat/SummarizerResult;)J + */ +JNIEXPORT jlong JNICALL Java_org_apache_spark_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL + (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/resources/log4j.properties b/mllib-dal/src/main/resources/log4j.properties index a33c21109..aec45c57e 100644 --- a/mllib-dal/src/main/resources/log4j.properties +++ b/mllib-dal/src/main/resources/log4j.properties @@ -1 +1 @@ -log4j.logger.org.apache.spark.ml.util.LibLoader=DEBUG +log4j.logger.com.intel.oap.mllib.LibLoader=DEBUG diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneCCL.scala similarity index 92% rename from mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/OneCCL.scala index 643ed8f54..0f00a31d8 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneCCL.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.util +package com.intel.oap.mllib import org.apache.spark.internal.Logging @@ -58,12 +58,15 @@ object OneCCL extends Logging { c_getAvailPort(localIP) } - @native private def c_init(size: Int, rank: Int, ip_port: String, param: CCLParam) : Int - @native private def c_cleanup() : Unit + @native private def c_init(size: Int, rank: Int, ip_port: String, param: CCLParam): Int - @native def isRoot() : Boolean - @native def rankID() : Int + @native private def c_cleanup(): Unit + + @native def isRoot(): Boolean + + @native def rankID(): Int @native def setEnv(key: String, value: String, overwrite: Boolean = true): Int + @native def c_getAvailPort(localIP: String): Int } diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala similarity index 99% rename from mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala index 9ae7d3fef..b5c429cb1 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package org.apache.spark.ml.util +package com.intel.oap.mllib import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, NumericTable, RowMergedNumericTable, Matrix => DALMatrix} import com.intel.daal.services.DaalContext @@ -54,7 +54,7 @@ object OneDAL { matrix } - + def numericTableToOldMatrix(table: NumericTable): OldMatrix = { val numRows = table.getNumberOfRows.toInt val numCols = table.getNumberOfColumns.toInt @@ -160,7 +160,7 @@ object OneDAL { doublesTables } - + def rddLabeledPointToSparseTables(labeledPoints: Dataset[_], executorNum: Int): RDD[(Long, Long)] = { require(executorNum > 0) @@ -395,7 +395,7 @@ object OneDAL { matrix } - def makeNumericTable (arrayVectors: Array[Vector]): NumericTable = { + def makeNumericTable(arrayVectors: Array[Vector]): NumericTable = { val numCols = arrayVectors.head.size val numRows: Int = arrayVectors.size @@ -422,7 +422,7 @@ object OneDAL { Iterator(table) } } - + def rddVectorToMergedTables(vectors: RDD[Vector], executorNum: Int): RDD[Long] = { require(executorNum > 0) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala similarity index 83% rename from mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala index 702d91774..df9640fa5 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/Utils.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala @@ -1,11 +1,12 @@ /* - * Copyright 2020 Intel Corporation + * 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 * - * 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 + * 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, @@ -14,29 +15,34 @@ * limitations under the License. */ -package org.apache.spark.ml.util +package com.intel.oap.mllib -import java.net.InetAddress -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.linalg.Vector import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession +import org.apache.spark.{SparkConf, SparkContext} + +import java.net.InetAddress object Utils { def isOAPEnabled(): Boolean = { val sc = SparkSession.active.sparkContext + return sc.getConf.getBoolean("spark.oap.mllib.enabled", true) + } + + def isOAPEnabled(sc: SparkContext): Boolean = { return sc.conf.getBoolean("spark.oap.mllib.enabled", true) } def getOneCCLIPPort(data: RDD[_]): String = { val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) - val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", + val kvsIP = data.sparkContext.getConf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) // TODO: right now we use a configured port, will optimize auto port detection // val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) val kvsPortDetected = 3000 - val kvsPort = data.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", + val kvsPort = data.sparkContext.getConf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) kvsIP + "_" + kvsPort diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesDALImpl.scala similarity index 84% rename from mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesDALImpl.scala index b886ee202..88029fc96 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayesDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesDALImpl.scala @@ -14,21 +14,30 @@ * limitations under the License. */ -package org.apache.spark.ml.classification +package com.intel.oap.mllib.classification +import com.intel.oap.mllib.Utils.getOneCCLIPPort +import com.intel.oap.mllib.{OneCCL, OneDAL} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -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.ml.classification.NaiveBayesModel +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector} +import org.apache.spark.ml.util.Instrumentation import org.apache.spark.sql.Dataset +class NaiveBayesDALModel private[mllib] ( + val uid: String, + val pi: Vector, + val theta: Matrix, + val sigma: Matrix +) + class NaiveBayesDALImpl(val uid: String, val classNum: Int, val executorNum: Int, val executorCores: Int ) extends Serializable with Logging { - def train(labeledPoints: Dataset[_], - instr: Option[Instrumentation]): NaiveBayesModel = { + def train(labeledPoints: Dataset[_]): NaiveBayesDALModel = { val kvsIPPort = getOneCCLIPPort(labeledPoints.rdd) @@ -82,7 +91,7 @@ class NaiveBayesDALImpl(val uid: String, assert(results.length == 1) val result = results(0) - val model = new NaiveBayesModel(uid, + val model = new NaiveBayesDALModel(uid, result._1, result._2, Matrices.zeros(0, 0)) diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesShim.scala new file mode 100644 index 000000000..53bb2a51d --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/classification/NaiveBayesShim.scala @@ -0,0 +1,40 @@ +/* + * 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 com.intel.oap.mllib.classification + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.classification.NaiveBayesModel +import org.apache.spark.ml.classification.spark320.{NaiveBayes => NaiveBayesSpark320} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.sql.Dataset +import org.apache.spark.{SPARK_VERSION, SparkException} + +trait NaiveBayesShim extends Logging { + def initShim(params: ParamMap): Unit + def train(dataset: Dataset[_]): NaiveBayesModel +} + +object NaiveBayesShim extends Logging { + def create(uid: String): NaiveBayesShim = { + logInfo(s"Loading NaiveBayes for Spark $SPARK_VERSION") + val shim = SPARK_VERSION match { + case "3.1.1" | "3.1.2" | "3.2.0" => new NaiveBayesSpark320(uid) + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + shim + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansDALImpl.scala similarity index 88% rename from mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansDALImpl.scala index 5fb340e08..44b508e1a 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansDALImpl.scala @@ -14,14 +14,13 @@ * limitations under the License. */ -package org.apache.spark.ml.clustering +package com.intel.oap.mllib.clustering -import com.intel.daal.data_management.data.{NumericTable, RowMergedNumericTable, Matrix => DALMatrix} -import com.intel.daal.services.DaalContext +import com.intel.oap.mllib.Utils.getOneCCLIPPort +import com.intel.oap.mllib.{OneCCL, OneDAL} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.util.Utils.getOneCCLIPPort import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeansModel => MLlibKMeansModel} import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} @@ -36,14 +35,14 @@ class KMeansDALImpl(var nClusters: Int, val executorCores: Int ) extends Serializable with Logging { - def train(data: RDD[Vector], instr: Option[Instrumentation]): MLlibKMeansModel = { + def train(data: RDD[Vector]): MLlibKMeansModel = { val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) val kvsIPPort = getOneCCLIPPort(coalescedTables) val sparkContext = data.sparkContext - val useGPU = sparkContext.conf.getBoolean("spark.oap.mllib.useGPU", false) + val useGPU = sparkContext.getConf.getBoolean("spark.oap.mllib.useGPU", false) val results = coalescedTables.mapPartitionsWithIndex { (rank, table) => @@ -99,7 +98,7 @@ class KMeansDALImpl(var nClusters: Int, } logInfo(s"The cost is $totalCost.") - instr.foreach(_.logInfo(s"OneDAL output centroids:\n${centerVectors.mkString("\n")}")) + logInfo(s"OneDAL output centroids:\n${centerVectors.mkString("\n")}") val parentModel = new MLlibKMeansModel( centerVectors.map(OldVectors.fromML(_)), diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansShim.scala new file mode 100644 index 000000000..8b3b01ee9 --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/clustering/KMeansShim.scala @@ -0,0 +1,40 @@ +/* + * 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 com.intel.oap.mllib.clustering + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.clustering.{KMeans, KMeansModel} +import org.apache.spark.ml.clustering.spark320.{KMeans => KMeansSpark320} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.sql.Dataset +import org.apache.spark.{SPARK_VERSION, SparkException} + +trait KMeansShim extends Logging { + def initShim(params: ParamMap): Unit + def fit(dataset: Dataset[_]): KMeansModel +} + +object KMeansShim extends Logging { + def create(uid: String): KMeansShim = { + logInfo(s"Loading KMeans for Spark $SPARK_VERSION") + val kmeans = SPARK_VERSION match { + case "3.1.1" | "3.1.2" | "3.2.0" => new KMeansSpark320(uid) + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + kmeans + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCADALImpl.scala similarity index 81% rename from mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCADALImpl.scala index 047b8057a..7796e671e 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCADALImpl.scala @@ -14,25 +14,32 @@ * limitations under the License. */ -package org.apache.spark.ml.feature +package com.intel.oap.mllib.feature -import java.util.Arrays import com.intel.daal.data_management.data.{HomogenNumericTable, NumericTable} +import com.intel.oap.mllib.Utils.getOneCCLIPPort +import com.intel.oap.mllib.{OneCCL, OneDAL} import org.apache.spark.TaskContext +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.ml.util.{OneCCL, OneDAL, Utils} import org.apache.spark.mllib.feature.{PCAModel => MLlibPCAModel, StandardScaler => MLlibStandardScaler} -import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.{DenseVector => OldDenseVector, DenseMatrix => OldDenseMatrix, Vectors => OldVectors} import org.apache.spark.rdd.RDD +import java.util.Arrays + +class PCADALModel private[mllib] ( + val k: Int, + val pc: OldDenseMatrix, + val explainedVariance: OldDenseVector) + class PCADALImpl(val k: Int, val executorNum: Int, val executorCores: Int) extends Serializable with Logging { - def train(data: RDD[Vector]): MLlibPCAModel = { + def train(data: RDD[Vector]): PCADALModel = { val normalizedData = normalizeData(data) @@ -41,7 +48,7 @@ class PCADALImpl(val k: Int, val kvsIPPort = getOneCCLIPPort(coalescedTables) val sparkContext = data.sparkContext - val useGPU = sparkContext.conf.getBoolean("spark.oap.mllib.useGPU", false) + val useGPU = sparkContext.getConf.getBoolean("spark.oap.mllib.useGPU", false) val results = coalescedTables.mapPartitionsWithIndex { (rank, table) => val gpuIndices = if (useGPU) { @@ -90,7 +97,7 @@ class PCADALImpl(val k: Int, val pc = results(0)._1 val explainedVariance = results(0)._2 - val parentModel = new MLlibPCAModel(k, + val parentModel = new PCADALModel(k, OldDenseMatrix.fromML(pc), OldVectors.fromML(explainedVariance).toDense ) @@ -115,15 +122,18 @@ class PCADALImpl(val k: Int, val numCols = table.getNumberOfColumns.toInt - val result = DenseMatrix.zeros(numCols, k) - - for (row <- 0 until k) { - for (col <- 0 until numCols) { - result(col, row) = data(row * numCols + col) - } - } - - result + // Column-major, transpose of top K rows of NumericTable + new DenseMatrix(numCols, k, data.slice(0, numCols * k), false) + +// val result = DenseMatrix.zeros(numCols, k) +// +// for (row <- 0 until k) { +// for (col <- 0 until numCols) { +// result(col, row) = data(row * numCols + col) +// } +// } +// +// result } private def getExplainedVarianceFromDAL(table_1xn: NumericTable, k: Int): DenseVector = { diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCAShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCAShim.scala new file mode 100644 index 000000000..1efe9d870 --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/feature/PCAShim.scala @@ -0,0 +1,40 @@ +/* + * 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 com.intel.oap.mllib.feature + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.PCAModel +import org.apache.spark.ml.feature.spark320.{PCA => PCASpark320} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.sql.Dataset +import org.apache.spark.{SPARK_VERSION, SparkException} + +trait PCAShim extends Logging { + def initShim(params: ParamMap): Unit + def fit(dataset: Dataset[_]): PCAModel +} + +object PCAShim extends Logging { + def create(uid: String): PCAShim = { + logInfo(s"Loading PCA for Spark $SPARK_VERSION") + val pca = SPARK_VERSION match { + case "3.1.1" | "3.1.2" | "3.2.0" => new PCASpark320(uid) + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + pca + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSDALImpl.scala similarity index 98% rename from mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSDALImpl.scala index c410ccda9..d920db537 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSDALImpl.scala @@ -14,20 +14,21 @@ * limitations under the License. */ -package org.apache.spark.ml.recommendation +package com.intel.oap.mllib.recommendation -import java.nio.{ByteBuffer, ByteOrder, FloatBuffer} -import scala.collection.mutable.ArrayBuffer -import scala.reflect.ClassTag import com.intel.daal.data_management.data.CSRNumericTable import com.intel.daal.services.DaalContext +import com.intel.oap.mllib.Utils.getOneCCLIPPort +import com.intel.oap.mllib.{OneCCL, OneDAL, Utils} import org.apache.spark.Partitioner import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.ALS.Rating -import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.ml.util._ import org.apache.spark.rdd.RDD +import java.nio.{ByteBuffer, ByteOrder, FloatBuffer} +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + class ALSDataPartitioner(blocks: Int, itemsInBlock: Long) extends Partitioner { def numPartitions: Int = blocks diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSShim.scala new file mode 100644 index 000000000..b46a0079d --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/recommendation/ALSShim.scala @@ -0,0 +1,56 @@ +/* + * 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 com.intel.oap.mllib.recommendation + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.recommendation.ALS.Rating +import org.apache.spark.ml.recommendation.spark312.{ALS => ALSSpark312} +import org.apache.spark.ml.recommendation.spark320.{ALS => ALSSpark320} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.{SPARK_VERSION, SparkException} + +import scala.reflect.ClassTag + +trait ALSShim extends Serializable with Logging { + def train[ID: ClassTag]( // scalastyle:ignore + ratings: RDD[Rating[ID]], + rank: Int, + numUserBlocks: Int, + numItemBlocks: Int, + maxIter: Int, + regParam: Double, + implicitPrefs: Boolean, + alpha: Double, + nonnegative: Boolean, + intermediateRDDStorageLevel: StorageLevel, + finalRDDStorageLevel: StorageLevel, + checkpointInterval: Int, + seed: Long)(implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) +} + +object ALSShim extends Logging { + def create(): ALSShim = { + logInfo(s"Loading ALS for Spark $SPARK_VERSION") + val als = SPARK_VERSION match { + case "3.1.1" | "3.1.2" => new ALSSpark312() + case "3.2.0" => new ALSSpark320() + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + als + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegressionDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionDALImpl.scala similarity index 87% rename from mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegressionDALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionDALImpl.scala index 73ae9be3d..b673094cf 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegressionDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionDALImpl.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.ml.regression +package com.intel.oap.mllib.regression +import com.intel.oap.mllib.Utils.getOneCCLIPPort +import com.intel.oap.mllib.{OneCCL, OneDAL} import org.apache.spark.internal.Logging import org.apache.spark.ml.linalg.{DenseVector, Vector} -import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.ml.util.{Instrumentation, OneCCL, OneDAL} +import org.apache.spark.ml.util.Instrumentation import org.apache.spark.sql.Dataset /** @@ -33,10 +34,10 @@ import org.apache.spark.sql.Dataset */ // LinearRegressionDALModel is the same with WeightedLeastSquaresModel // diagInvAtWA and objectiveHistory are not supported right now -private[ml] class LinearRegressionDALModel(val coefficients: DenseVector, - val intercept: Double, - val diagInvAtWA: DenseVector, - val objectiveHistory: Array[Double]) +private[mllib] class LinearRegressionDALModel(val coefficients: DenseVector, + val intercept: Double, + val diagInvAtWA: DenseVector, + val objectiveHistory: Array[Double]) extends Serializable { } @@ -56,8 +57,7 @@ class LinearRegressionDALImpl( val fitIntercept: Boolean, /** * Creates a [[LinearRegressionDALModel]] from an RDD of [[Vector]]s. */ - def train(labeledPoints: Dataset[_], - instr: Option[Instrumentation]): LinearRegressionDALModel = { + def train(labeledPoints: Dataset[_]): LinearRegressionDALModel = { val kvsIPPort = getOneCCLIPPort(labeledPoints.rdd) diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionShim.scala new file mode 100644 index 000000000..68a250073 --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/regression/LinearRegressionShim.scala @@ -0,0 +1,42 @@ +/* + * 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 com.intel.oap.mllib.regression + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.regression.LinearRegressionModel +import org.apache.spark.ml.regression.spark312.{LinearRegression => LinearRegressionSpark312} +import org.apache.spark.ml.regression.spark320.{LinearRegression => LinearRegressionSpark320} +import org.apache.spark.sql.Dataset +import org.apache.spark.{SPARK_VERSION, SparkException} + +trait LinearRegressionShim extends Serializable with Logging { + def initShim(params: ParamMap): Unit + def train(dataset: Dataset[_]): LinearRegressionModel +} + +object LinearRegressionShim extends Logging { + def create(uid: String): LinearRegressionShim = { + logInfo(s"Loading ALS for Spark $SPARK_VERSION") + val linearRegression = SPARK_VERSION match { + case "3.1.1" | "3.1.2" => new LinearRegressionSpark312(uid) + case "3.2.0" => new LinearRegressionSpark320(uid) + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + linearRegression + } +} \ No newline at end of file diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/stat/CorrelationDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationDALImpl.scala similarity index 90% rename from mllib-dal/src/main/scala/org/apache/spark/ml/stat/CorrelationDALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationDALImpl.scala index 146c8fcd0..70b782da8 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/stat/CorrelationDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationDALImpl.scala @@ -14,14 +14,13 @@ * limitations under the License. */ -package org.apache.spark.ml.stat +package com.intel.oap.mllib.stat -import java.util.Arrays +import com.intel.oap.mllib.Utils.getOneCCLIPPort +import com.intel.oap.mllib.{OneCCL, OneDAL} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging -import org.apache.spark.ml.util.{OneCCL, OneDAL, Utils} -import org.apache.spark.ml.linalg.{Vector, Matrix} -import org.apache.spark.ml.util.Utils.getOneCCLIPPort +import org.apache.spark.ml.linalg.{Matrix, Vector} import org.apache.spark.rdd.RDD class CorrelationDALImpl( @@ -34,7 +33,7 @@ class CorrelationDALImpl( val kvsIPPort = getOneCCLIPPort(data) val sparkContext = data.sparkContext - val useGPU = sparkContext.conf.getBoolean("spark.oap.mllib.useGPU", false) + val useGPU = sparkContext.getConf.getBoolean("spark.oap.mllib.useGPU", false) val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationShim.scala new file mode 100644 index 000000000..a616dcb90 --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/CorrelationShim.scala @@ -0,0 +1,43 @@ +/* + * 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 com.intel.oap.mllib.stat + +import org.apache.spark.{SPARK_VERSION, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.recommendation.ALS.Rating +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.storage.StorageLevel + +import scala.reflect.ClassTag + +import org.apache.spark.ml.stat.spark320.{Correlation => CorrelationSpark320 } + +trait CorrelationShim extends Serializable with Logging { + def corr(dataset: Dataset[_], column: String, method: String): DataFrame +} + +object CorrelationShim extends Logging { + def create(): CorrelationShim = { + logInfo(s"Loading Correlation for Spark $SPARK_VERSION") + val als = SPARK_VERSION match { + case "3.1.1" | "3.1.2" | "3.2.0" => new CorrelationSpark320() + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + als + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala new file mode 100644 index 000000000..74de47a85 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -0,0 +1,130 @@ +/* + * 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 com.intel.oap.mllib.classification.NaiveBayesShim +import org.apache.spark.annotation.Since +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util._ +import org.apache.spark.sql.Dataset + +// 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 = { + val shim = NaiveBayesShim.create(uid) + shim.initShim(extractParamMap()) + shim.train(dataset) + } + + @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) +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/classification/spark320/NaiveBayes.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/spark320/NaiveBayes.scala new file mode 100644 index 000000000..79340fa4e --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/classification/spark320/NaiveBayes.scala @@ -0,0 +1,332 @@ +/* + * 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.spark320 + +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.classification.{NaiveBayesDALImpl, NaiveBayesShim} +import org.apache.spark.annotation.Since +import org.apache.spark.ml.functions.checkNonNegativeWeight +import org.apache.spark.ml.linalg._ +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.sql.Dataset +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.ml.classification.{NaiveBayes => SparkNaiveBayes} +import org.apache.spark.ml.classification._ +import org.apache.spark.ml.param.ParamMap + +// 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 SparkNaiveBayes with NaiveBayesShim { + + import SparkNaiveBayes._ + import NaiveBayes._ + + @Since("1.5.0") + def this() = this(Identifiable.randomUID("nb")) + + override def initShim(params: ParamMap): Unit = { + params.toSeq.foreach { paramMap.put(_) } + } + override 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 sc = dataset.sparkSession.sparkContext + val model = if (Utils.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) + } + model + 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 spark = dataset.sparkSession + import spark.implicits._ + + val sc = spark.sparkContext + + val executor_num = Utils.sparkExecutorNum(sc) + val executor_cores = Utils.sparkExecutorCores() + + logInfo(s"NaiveBayesDAL fit using $executor_num Executors") + + // 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) + + // 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 + } + + instr.logNumClasses(numClasses) + + val labeledPointsDS = dataset + .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) + + val dalModel = new NaiveBayesDALImpl(uid, numClasses, + executor_num, executor_cores).train(labeledPointsDS) + + val model = copyValues(new NaiveBayesModel( + dalModel.uid, dalModel.pi, dalModel.theta, dalModel.sigma)) + + // Set labels to be compatible with old mllib model + val labels = (0 until numClasses).map(_.toDouble).toArray + model.setOldLabels(labels) + + 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) { + checkNonNegativeWeight(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) { + checkNonNegativeWeight(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) + } +} \ No newline at end of file diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala new file mode 100644 index 000000000..d960cb12b --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -0,0 +1,109 @@ +/* + * 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.clustering + +import com.intel.oap.mllib.clustering.KMeansShim +import org.apache.spark.annotation.Since +import org.apache.spark.ml.Estimator +import org.apache.spark.ml.param._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.types.StructType + +/** + * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. + * + * @see Bahmani et al., Scalable k-means++. + */ +@Since("1.5.0") +class KMeans @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { + + @Since("1.5.0") + override def copy(extra: ParamMap): KMeans = defaultCopy(extra) + + @Since("1.5.0") + def this() = this(Identifiable.randomUID("kmeans")) + + /** @group setParam */ + @Since("1.5.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("1.5.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + @Since("1.5.0") + def setK(value: Int): this.type = set(k, value) + + /** @group expertSetParam */ + @Since("1.5.0") + def setInitMode(value: String): this.type = set(initMode, value) + + /** @group expertSetParam */ + @Since("2.4.0") + def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) + + /** @group expertSetParam */ + @Since("1.5.0") + def setInitSteps(value: Int): this.type = set(initSteps, value) + + /** @group setParam */ + @Since("1.5.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ + @Since("1.5.0") + def setTol(value: Double): this.type = set(tol, value) + + /** @group setParam */ + @Since("1.5.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** + * 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("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): KMeansModel = { + val shim = KMeansShim.create(uid) + shim.initShim(extractParamMap()) + shim.fit(dataset) + } + + @Since("1.5.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} + +@Since("1.6.0") +object KMeans extends DefaultParamsReadable[KMeans] { + + @Since("1.6.0") + override def load(path: String): KMeans = super.load(path) +} + diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/spark320/KMeans.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/spark320/KMeans.scala new file mode 100644 index 000000000..8031f8fc3 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/spark320/KMeans.scala @@ -0,0 +1,189 @@ +/* + * 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.clustering.spark320 + +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.clustering.{KMeansDALImpl, KMeansShim} +import org.apache.spark.annotation.Since +import org.apache.spark.ml.clustering.{KMeans => SparkKMeans, _} +import org.apache.spark.ml.functions.checkNonNegativeWeight +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.param._ +import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.clustering.{DistanceMeasure, VectorWithNorm, KMeans => MLlibKMeans} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DoubleType +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.storage.StorageLevel + +/** + * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. + * + * @see Bahmani et al., Scalable k-means++. + */ +@Since("1.5.0") +class KMeans @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends SparkKMeans with KMeansShim { + + override def initShim(params: ParamMap): Unit = { + params.toSeq.foreach { paramMap.put(_) } + } + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): KMeansModel = instrumented { instr => + transformSchema(dataset.schema, logging = true) + + instr.logPipelineStage(this) + instr.logDataset(dataset) + instr.logParams(this, featuresCol, predictionCol, k, initMode, initSteps, distanceMeasure, + maxIter, seed, tol, weightCol) + + val handleWeight = isDefined(weightCol) && $(weightCol).nonEmpty + val w = if (handleWeight) { + checkNonNegativeWeight(col($(weightCol)).cast(DoubleType)) + } else { + lit(1.0) + } + val instances = dataset.select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w) + .rdd.map { case Row(point: Vector, weight: Double) => (point, weight) } + + val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) + + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + dataset.sparkSession.sparkContext) + val useKMeansDAL = Utils.isOAPEnabled() && isPlatformSupported && + $(distanceMeasure) == "euclidean" && !handleWeight + + val model = if (useKMeansDAL) { + trainWithDAL(instances, handlePersistence) + } else { + trainWithML(instances, handlePersistence) + } + + val summary = new KMeansSummary( + model.transform(dataset), + $(predictionCol), + $(featuresCol), + $(k), + model.parentModel.numIter, + model.parentModel.trainingCost) + + model.setSummary(Some(summary)) + instr.logNamedValue("clusterSizes", summary.clusterSizes) + + model + } + + private def trainWithDAL(instances: RDD[(Vector, Double)], + handlePersistence: Boolean): KMeansModel = instrumented { instr => + + val sc = instances.sparkContext + + val executor_num = Utils.sparkExecutorNum(sc) + val executor_cores = Utils.sparkExecutorCores() + + logInfo(s"KMeansDAL fit using $executor_num Executors") + + val initStartTime = System.nanoTime() + + val distanceMeasureInstance = DistanceMeasure.decodeFromString($(distanceMeasure)) + + // Use MLlibKMeans to initialize centers + val mllibKMeans = new MLlibKMeans() + .setK($(k)) + .setInitializationMode($(initMode)) + .setInitializationSteps($(initSteps)) + .setMaxIterations($(maxIter)) + .setSeed($(seed)) + .setEpsilon($(tol)) + .setDistanceMeasure($(distanceMeasure)) + + val dataWithNorm = instances.map { + case (point: Vector, weight: Double) => new VectorWithNorm(point) + } + + // Cache for init + dataWithNorm.persist(StorageLevel.MEMORY_AND_DISK) + + val centersWithNorm = if ($(initMode) == "random") { + mllibKMeans.initRandom(dataWithNorm) + } else { + mllibKMeans.initKMeansParallel(dataWithNorm, distanceMeasureInstance) + } + + dataWithNorm.unpersist() + + val centers = centersWithNorm.map(_.vector) + + val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 + + val strInitMode = $(initMode) + logInfo(f"Initialization with $strInitMode took $initTimeInSeconds%.3f seconds.") + + if (handlePersistence) { + instances.persist(StorageLevel.MEMORY_AND_DISK) + } + + val inputData = instances.map { + case (point: Vector, weight: Double) => point + } + + val kmeansDAL = new KMeansDALImpl(getK, getMaxIter, getTol, + DistanceMeasure.EUCLIDEAN, centers, executor_num, executor_cores) + + val parentModel = kmeansDAL.train(inputData) + + val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) + + if (handlePersistence) { + instances.unpersist() + } + + model + } + + private def trainWithML(instances: RDD[(Vector, Double)], + handlePersistence: Boolean): KMeansModel = instrumented { instr => + val oldVectorInstances = instances.map { + case (point: Vector, weight: Double) => (OldVectors.fromML(point), weight) + } + val algo = new MLlibKMeans() + .setK($(k)) + .setInitializationMode($(initMode)) + .setInitializationSteps($(initSteps)) + .setMaxIterations($(maxIter)) + .setSeed($(seed)) + .setEpsilon($(tol)) + .setDistanceMeasure($(distanceMeasure)) + val parentModel = algo.runWithWeight(oldVectorInstances, handlePersistence, Some(instr)) + val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) + model + } +} + +@Since("1.6.0") +object KMeans extends DefaultParamsReadable[KMeans] { + + @Since("1.6.0") + override def load(path: String): KMeans = super.load(path) +} \ No newline at end of file diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCA.scala new file mode 100644 index 000000000..1d91323d8 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -0,0 +1,76 @@ +/* + * 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.feature + +import com.intel.oap.mllib.feature.PCAShim +import org.apache.spark.annotation.Since +import org.apache.spark.ml._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql._ +import org.apache.spark.sql.types.StructType + +/** + * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` + * principal components. + */ +@Since("1.5.0") +class PCA @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { + + @Since("1.5.0") + def this() = this(Identifiable.randomUID("pca")) + + /** @group setParam */ + @Since("1.5.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("1.5.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + /** @group setParam */ + @Since("1.5.0") + def setK(value: Int): this.type = set(k, value) + + /** + * Computes a [[PCAModel]] that contains the principal components of the input vectors. + */ + @Since("2.0.0") + override def fit(dataset: Dataset[_]): PCAModel = { + val shim = PCAShim.create(uid) + shim.initShim(extractParamMap()) + shim.fit(dataset) + } + + @Since("1.5.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + @Since("1.5.0") + override def copy(extra: ParamMap): PCA = defaultCopy(extra) +} + +@Since("1.6.0") +object PCA extends DefaultParamsReadable[PCA] { + + @Since("1.6.0") + override def load(path: String): PCA = super.load(path) +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/spark320/PCA.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/spark320/PCA.scala new file mode 100644 index 000000000..590712628 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/spark320/PCA.scala @@ -0,0 +1,80 @@ +/* + * 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.feature.spark320 + +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.feature.{PCADALImpl, PCAShim} +import org.apache.spark.annotation.Since +import org.apache.spark.ml.feature.{PCA => SparkPCA, _} +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.feature +import org.apache.spark.mllib.feature.{PCAModel => OldPCAModel} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.sql._ + +/** + * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` + * principal components. + */ +@Since("1.5.0") +class PCA @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends SparkPCA with PCAShim { + + override def initShim(params: ParamMap): Unit = { + params.toSeq.foreach { paramMap.put(_) } + } + + /** + * Computes a [[PCAModel]] that contains the principal components of the input vectors. + */ + @Since("2.0.0") + override def fit(dataset: Dataset[_]): PCAModel = { + transformSchema(dataset.schema, logging = true) + val input = dataset.select($(inputCol)).rdd + val inputVectors = input.map { + case Row(v: Vector) => v + } + + val numFeatures = inputVectors.first().size + require($(k) <= numFeatures, + s"source vector size $numFeatures must be no less than k=$k") + + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + dataset.sparkSession.sparkContext) + + // Call oneDAL Correlation PCA implementation when numFeatures < 65535 and fall back otherwise + val parentModel = if (numFeatures < 65535 && Utils.isOAPEnabled() && isPlatformSupported) { + val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val pca = new PCADALImpl(k = $(k), executor_num, executor_cores) + val pcaDALModel = pca.train(inputVectors) + new OldPCAModel(pcaDALModel.k, pcaDALModel.pc, pcaDALModel.explainedVariance) + } else { + val inputOldVectors = inputVectors.map { + case v: Vector => OldVectors.fromML(v) + } + val pca = new feature.PCA(k = $(k)) + val pcaModel = pca.fit(inputOldVectors) + pcaModel + } + copyValues(new PCAModel(uid, parentModel.pc.asML, parentModel.explainedVariance.asML) + .setParent(this)) + } +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala similarity index 60% rename from mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/recommendation/ALS.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 71195397b..2752ebf9b 100644 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -17,541 +17,31 @@ package org.apache.spark.ml.recommendation -import java.{util => ju} -import java.io.IOException -import java.util.Locale - -import scala.collection.mutable -import scala.reflect.ClassTag -import scala.util.{Sorting, Try} -import scala.util.hashing.byteswap64 - import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.hadoop.fs.Path -import org.json4s.DefaultFormats -import org.json4s.JsonDSL._ - -import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} +import com.intel.oap.mllib.recommendation.ALSShim import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.linalg.BLAS +import org.apache.spark.ml.Estimator import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Utils => DALImplUtils, _} import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.ml.util._ import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.{DeterministicLevel, RDD} -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.Utils import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} -/** - * Common params for ALS and ALSModel. - */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol - with HasBlockSize { - /** - * Param for the column name for user ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "user" - * @group param - */ - val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getUserCol: String = $(userCol) - - /** - * Param for the column name for item ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "item" - * @group param - */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getItemCol: String = $(itemCol) - - /** - * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is - * out of integer range or contains a fractional part. - */ - protected[recommendation] val checkedCast = udf { (n: Any) => - n match { - case v: Int => v // Avoid unnecessary casting - case v: Number => - val intV = v.intValue - // Checks if number within Int range and has no fractional part. - if (v.doubleValue == intV) { - intV - } else { - throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"and without fractional part for columns ${$(userCol)} and ${$(itemCol)}. " + - s"Value $n was either out of Integer range or contained a fractional part that " + - s"could not be converted.") - } - case _ => throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"for columns ${$(userCol)} and ${$(itemCol)}. Value $n was not numeric.") - } - } - - /** - * Param for strategy for dealing with unknown or new users/items at prediction time. - * This may be useful in cross-validation or production scenarios, for handling user/item ids - * the model has not seen in the training data. - * Supported values: - * - "nan": predicted value for unknown ids will be NaN. - * - "drop": rows in the input DataFrame containing unknown ids will be dropped from - * the output DataFrame containing predictions. - * Default: "nan". - * @group expertParam - */ - val coldStartStrategy = new Param[String](this, "coldStartStrategy", - "strategy for dealing with unknown or new users/items at prediction time. This may be " + - "useful in cross-validation or production scenarios, for handling user/item ids the model " + - "has not seen in the training data. Supported values: " + - s"${ALSModel.supportedColdStartStrategies.mkString(",")}.", - (s: String) => - ALSModel.supportedColdStartStrategies.contains(s.toLowerCase(Locale.ROOT))) - - /** @group expertGetParam */ - def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) - - setDefault(blockSize -> 4096) -} - -/** - * Common params for ALS. - */ -private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter with HasRegParam - with HasCheckpointInterval with HasSeed { - - /** - * Param for rank of the matrix factorization (positive). - * Default: 10 - * @group param - */ - val rank = new IntParam(this, "rank", "rank of the factorization", ParamValidators.gtEq(1)) - - /** @group getParam */ - def getRank: Int = $(rank) - - /** - * Param for number of user blocks (positive). - * Default: 10 - * @group param - */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumUserBlocks: Int = $(numUserBlocks) - - /** - * Param for number of item blocks (positive). - * Default: 10 - * @group param - */ - val numItemBlocks = new IntParam(this, "numItemBlocks", "number of item blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumItemBlocks: Int = $(numItemBlocks) - - /** - * Param to decide whether to use implicit preference. - * Default: false - * @group param - */ - val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") - - /** @group getParam */ - def getImplicitPrefs: Boolean = $(implicitPrefs) - - /** - * Param for the alpha parameter in the implicit preference formulation (nonnegative). - * Default: 1.0 - * @group param - */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", - ParamValidators.gtEq(0)) - - /** @group getParam */ - def getAlpha: Double = $(alpha) - - /** - * Param for the column name for ratings. - * Default: "rating" - * @group param - */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") - - /** @group getParam */ - def getRatingCol: String = $(ratingCol) - - /** - * Param for whether to apply nonnegativity constraints. - * Default: false - * @group param - */ - val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares") - - /** @group getParam */ - def getNonnegative: Boolean = $(nonnegative) - - /** - * Param for StorageLevel for intermediate datasets. Pass in a string representation of - * `StorageLevel`. Cannot be "NONE". - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") - - /** @group expertGetParam */ - def getIntermediateStorageLevel: String = $(intermediateStorageLevel) - - /** - * Param for StorageLevel for ALS model factors. Pass in a string representation of - * `StorageLevel`. - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val finalStorageLevel = new Param[String](this, "finalStorageLevel", - "StorageLevel for ALS model factors.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess) - - /** @group expertGetParam */ - def getFinalStorageLevel: String = $(finalStorageLevel) - - setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, - implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", - ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10, - intermediateStorageLevel -> "MEMORY_AND_DISK", finalStorageLevel -> "MEMORY_AND_DISK", - coldStartStrategy -> "nan") - - /** - * Validates and transforms the input schema. - * - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - // rating will be cast to Float - SchemaUtils.checkNumericType(schema, $(ratingCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } -} - -/** - * Model fitted by ALS. - * - * @param rank rank of the matrix factorization model - * @param userFactors a DataFrame that stores user factors in two columns: `id` and `features` - * @param itemFactors a DataFrame that stores item factors in two columns: `id` and `features` - */ -@Since("1.3.0") -class ALSModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("1.4.0") val rank: Int, - @transient val userFactors: DataFrame, - @transient val itemFactors: DataFrame) - extends Model[ALSModel] with ALSModelParams with MLWritable { - - /** @group setParam */ - @Since("1.4.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.4.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => - if (featuresA != null && featuresB != null) { - var dotProduct = 0.0f - var i = 0 - while (i < rank) { - dotProduct += featuresA(i) * featuresB(i) - i += 1 - } - dotProduct - } else { - Float.NaN - } - } - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema) - // create a new column named map(predictionCol) by running the predict UDF. - val predictions = dataset - .join(userFactors, - checkedCast(dataset($(userCol))) === userFactors("id"), "left") - .join(itemFactors, - checkedCast(dataset($(itemCol))) === itemFactors("id"), "left") - .select(dataset("*"), - predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) - getColdStartStrategy match { - case ALSModel.Drop => - predictions.na.drop("all", Seq($(predictionCol))) - case ALSModel.NaN => - predictions - } - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALSModel = { - val copied = new ALSModel(uid, rank, userFactors, itemFactors) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new ALSModel.ALSModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"ALSModel: uid=$uid, rank=$rank" - } - - /** - * Returns top `numItems` items recommended for each user, for all users. - * @param numItems max number of recommendations for each user - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numItems` items recommended for each user id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of user ids. The column name must match `userCol`. - * @param numItems max number of recommendations for each user. - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item, for all items. - * @param numUsers max number of recommendations for each item - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of item ids. The column name must match `itemCol`. - * @param numUsers max number of recommendations for each item. - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns a subset of a factor DataFrame limited to only those unique ids contained - * in the input dataset. - * @param dataset input Dataset containing id column to user to filter factors. - * @param factors factor DataFrame to filter. - * @param column column name containing the ids in the input dataset. - * @return DataFrame containing factors only for those ids present in both the input dataset and - * the factor DataFrame. - */ - private def getSourceFactorSubset( - dataset: Dataset[_], - factors: DataFrame, - column: String): DataFrame = { - factors - .join(dataset.select(column), factors("id") === dataset(column), joinType = "left_semi") - .select(factors("id"), factors("features")) - } - - /** - * Makes recommendations for all users (or items). - * - * Note: the previous approach used for computing top-k recommendations - * used a cross-join followed by predicting a score for each row of the joined dataset. - * However, this results in exploding the size of intermediate data. While Spark SQL makes it - * relatively efficient, the approach implemented here is significantly more efficient. - * - * This approach groups factors into blocks and computes the top-k elements per block, - * using dot product and an efficient [[BoundedPriorityQueue]] (instead of gemm). - * It then computes the global top-k by aggregating the per block top-k elements with - * a [[TopByKeyAggregator]]. This significantly reduces the size of intermediate and shuffle data. - * This is the DataFrame equivalent to the approach used in - * [[org.apache.spark.mllib.recommendation.MatrixFactorizationModel]]. - * - * @param srcFactors src factors for which to generate recommendations - * @param dstFactors dst factors used to make recommendations - * @param srcOutputColumn name of the column for the source ID in the output DataFrame - * @param dstOutputColumn name of the column for the destination ID in the output DataFrame - * @param num max number of recommendations for each record - * @return a DataFrame of (srcOutputColumn: Int, recommendations), where recommendations are - * stored as an array of (dstOutputColumn: Int, rating: Float) Rows. - */ - private def recommendForAll( - srcFactors: DataFrame, - dstFactors: DataFrame, - srcOutputColumn: String, - dstOutputColumn: String, - num: Int, - blockSize: Int): DataFrame = { - import srcFactors.sparkSession.implicits._ - - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) - val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) - .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] - .flatMap { case (srcIter, dstIter) => - val m = srcIter.size - val n = math.min(dstIter.size, num) - val output = new Array[(Int, Int, Float)](m * n) - var i = 0 - val pq = new BoundedPriorityQueue[(Int, Float)](num)(Ordering.by(_._2)) - srcIter.foreach { case (srcId, srcFactor) => - dstIter.foreach { case (dstId, dstFactor) => - // We use F2jBLAS which is faster than a call to native BLAS for vector dot product - val score = BLAS.f2jBLAS.sdot(rank, srcFactor, 1, dstFactor, 1) - pq += dstId -> score - } - pq.foreach { case (dstId, score) => - output(i) = (srcId, dstId, score) - i += 1 - } - pq.clear() - } - output.toSeq - } - // We'll force the IDs to be Int. Unfortunately this converts IDs to Int in the output. - val topKAggregator = new TopByKeyAggregator[Int, Int, Float](num, Ordering.by(_._2)) - val recs = ratings.as[(Int, Int, Float)].groupByKey(_._1).agg(topKAggregator.toColumn) - .toDF("id", "recommendations") - - val arrayType = ArrayType( - new StructType() - .add(dstOutputColumn, IntegerType) - .add("rating", FloatType) - ) - recs.select($"id".as(srcOutputColumn), $"recommendations".cast(arrayType)) - } - - /** - * Blockifies factors to improve the efficiency of cross join - */ - private def blockify( - factors: Dataset[(Int, Array[Float])], - blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { - import factors.sparkSession.implicits._ - factors.mapPartitions(_.grouped(blockSize)) - } - -} - -@Since("1.6.0") -object ALSModel extends MLReadable[ALSModel] { - - private val NaN = "nan" - private val Drop = "drop" - private[recommendation] final val supportedColdStartStrategies = Array(NaN, Drop) - - @Since("1.6.0") - override def read: MLReader[ALSModel] = new ALSModelReader - - @Since("1.6.0") - override def load(path: String): ALSModel = super.load(path) - - private[ALSModel] class ALSModelWriter(instance: ALSModel) extends MLWriter { - - override protected def saveImpl(path: String): Unit = { - val extraMetadata = "rank" -> instance.rank - DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) - val userPath = new Path(path, "userFactors").toString - instance.userFactors.write.format("parquet").save(userPath) - val itemPath = new Path(path, "itemFactors").toString - instance.itemFactors.write.format("parquet").save(itemPath) - } - } - - private class ALSModelReader extends MLReader[ALSModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[ALSModel].getName - - override def load(path: String): ALSModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - implicit val format = DefaultFormats - val rank = (metadata.metadata \ "rank").extract[Int] - val userPath = new Path(path, "userFactors").toString - val userFactors = sparkSession.read.format("parquet").load(userPath) - val itemPath = new Path(path, "itemFactors").toString - val itemFactors = sparkSession.read.format("parquet").load(itemPath) - - val model = new ALSModel(metadata.uid, rank, userFactors, itemFactors) - - metadata.getAndSetParams(model) - model - } - } -} +import java.{util => ju} +import scala.collection.mutable +import scala.reflect.ClassTag +import scala.util.Sorting +import scala.util.hashing.byteswap64 /** * Alternating Least Squares (ALS) matrix factorization. @@ -918,199 +408,22 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { checkpointInterval: Int = 10, seed: Long = 0L)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - - val isPlatformSupported = DALImplUtils.checkClusterPlatformCompatibility(ratings.sparkContext) - - val (userIdAndFactors, itemIdAndFactors) = - if (implicitPrefs && DALImplUtils.isOAPEnabled() && isPlatformSupported) { - new ALSDALImpl(ratings, rank, maxIter, regParam, alpha, seed).train() - } else { - trainMLlib(ratings, rank, numUserBlocks, numItemBlocks, maxIter, regParam, implicitPrefs, - alpha, nonnegative, intermediateRDDStorageLevel, finalRDDStorageLevel, - checkpointInterval, seed) - } - - (userIdAndFactors, itemIdAndFactors) - } - - private def trainDAL[ID: ClassTag]( - ratings: RDD[Rating[ID]], - rank: Int, - maxIter: Int, - regParam: Double, - alpha: Double, - seed: Long): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - null - } - - /** - * Implementation of the ALS algorithm. - * - * This implementation of the ALS factorization algorithm partitions the two sets of factors among - * Spark workers so as to reduce network communication by only sending one copy of each factor - * vector to each Spark worker on each iteration, and only if needed. This is achieved by - * precomputing some information about the ratings matrix to determine which users require which - * item factors and vice versa. See the Scaladoc for `InBlock` for a detailed explanation of how - * the precomputation is done. - * - * In addition, since each iteration of calculating the factor matrices depends on the known - * ratings, which are spread across Spark partitions, a naive implementation would incur - * significant network communication overhead between Spark workers, as the ratings RDD would be - * repeatedly shuffled during each iteration. This implementation reduces that overhead by - * performing the shuffling operation up front, precomputing each partition's ratings dependencies - * and duplicating those values to the appropriate workers before starting iterations to solve for - * the factor matrices. See the Scaladoc for `OutBlock` for a detailed explanation of how the - * precomputation is done. - * - * Note that the term "rating block" is a bit of a misnomer, as the ratings are not partitioned by - * contiguous blocks from the ratings matrix but by a hash function on the rating's location in - * the matrix. If it helps you to visualize the partitions, it is easier to think of the term - * "block" as referring to a subset of an RDD containing the ratings rather than a contiguous - * submatrix of the ratings matrix. - */ - private def trainMLlib[ID: ClassTag]( // scalastyle:ignore - ratings: RDD[Rating[ID]], - rank: Int = 10, - numUserBlocks: Int = 10, - numItemBlocks: Int = 10, - maxIter: Int = 10, - regParam: Double = 0.1, - implicitPrefs: Boolean = false, - alpha: Double = 1.0, - nonnegative: Boolean = false, - intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, - finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, - checkpointInterval: Int = 10, - seed: Long = 0L)( - implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - - require(!ratings.isEmpty(), s"No ratings available from $ratings") - require(intermediateRDDStorageLevel != StorageLevel.NONE, - "ALS is not designed to run without persisting intermediate RDDs.") - - val sc = ratings.sparkContext - - // Precompute the rating dependencies of each partition - val userPart = new ALSPartitioner(numUserBlocks) - val itemPart = new ALSPartitioner(numItemBlocks) - val blockRatings = partitionRatings(ratings, userPart, itemPart) - .persist(intermediateRDDStorageLevel) - val (userInBlocks, userOutBlocks) = - makeBlocks("user", blockRatings, userPart, itemPart, intermediateRDDStorageLevel) - userOutBlocks.count() // materialize blockRatings and user blocks - val swappedBlockRatings = blockRatings.map { - case ((userBlockId, itemBlockId), RatingBlock(userIds, itemIds, localRatings)) => - ((itemBlockId, userBlockId), RatingBlock(itemIds, userIds, localRatings)) - } - val (itemInBlocks, itemOutBlocks) = - makeBlocks("item", swappedBlockRatings, itemPart, userPart, intermediateRDDStorageLevel) - itemOutBlocks.count() // materialize item blocks - - // Encoders for storing each user/item's partition ID and index within its partition using a - // single integer; used as an optimization - val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) - val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) - - // These are the user and item factor matrices that, once trained, are multiplied together to - // estimate the rating matrix. The two matrices are stored in RDDs, partitioned by column such - // that each factor column resides on the same Spark worker as its corresponding user or item. - val seedGen = new XORShiftRandom(seed) - var userFactors = initialize(userInBlocks, rank, seedGen.nextLong()) - var itemFactors = initialize(itemInBlocks, rank, seedGen.nextLong()) - - val solver = if (nonnegative) new NNLSSolver else new CholeskySolver - - var previousCheckpointFile: Option[String] = None - val shouldCheckpoint: Int => Boolean = (iter) => - sc.checkpointDir.isDefined && checkpointInterval != -1 && (iter % checkpointInterval == 0) - val deletePreviousCheckpointFile: () => Unit = () => - previousCheckpointFile.foreach { file => - try { - val checkpointFile = new Path(file) - checkpointFile.getFileSystem(sc.hadoopConfiguration).delete(checkpointFile, true) - } catch { - case e: IOException => - logWarning(s"Cannot delete checkpoint file $file:", e) - } - } - - if (implicitPrefs) { - for (iter <- 1 to maxIter) { - userFactors.setName(s"userFactors-$iter").persist(intermediateRDDStorageLevel) - val previousItemFactors = itemFactors - itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder, implicitPrefs, alpha, solver) - previousItemFactors.unpersist() - itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) - // TODO: Generalize PeriodicGraphCheckpointer and use it here. - val deps = itemFactors.dependencies - if (shouldCheckpoint(iter)) { - itemFactors.checkpoint() // itemFactors gets materialized in computeFactors - } - val previousUserFactors = userFactors - userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder, implicitPrefs, alpha, solver) - if (shouldCheckpoint(iter)) { - ALS.cleanShuffleDependencies(sc, deps) - deletePreviousCheckpointFile() - previousCheckpointFile = itemFactors.getCheckpointFile - } - previousUserFactors.unpersist() - } - } else { - var previousCachedItemFactors: Option[RDD[(Int, FactorBlock)]] = None - for (iter <- 0 until maxIter) { - itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder, solver = solver) - if (shouldCheckpoint(iter)) { - itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) - val deps = itemFactors.dependencies - itemFactors.checkpoint() - itemFactors.count() // checkpoint item factors and cut lineage - ALS.cleanShuffleDependencies(sc, deps) - deletePreviousCheckpointFile() - - previousCachedItemFactors.foreach(_.unpersist()) - previousCheckpointFile = itemFactors.getCheckpointFile - previousCachedItemFactors = Option(itemFactors) - } - userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder, solver = solver) - } - } - val userIdAndFactors = userInBlocks - .mapValues(_.srcIds) - .join(userFactors) - .mapPartitions({ items => - items.flatMap { case (_, (ids, factors)) => - ids.view.zip(factors) - } - // Preserve the partitioning because IDs are consistent with the partitioners in userInBlocks - // and userFactors. - }, preservesPartitioning = true) - .setName("userFactors") - .persist(finalRDDStorageLevel) - val itemIdAndFactors = itemInBlocks - .mapValues(_.srcIds) - .join(itemFactors) - .mapPartitions({ items => - items.flatMap { case (_, (ids, factors)) => - ids.view.zip(factors) - } - }, preservesPartitioning = true) - .setName("itemFactors") - .persist(finalRDDStorageLevel) - if (finalRDDStorageLevel != StorageLevel.NONE) { - userIdAndFactors.count() - userInBlocks.unpersist() - userOutBlocks.unpersist() - itemOutBlocks.unpersist() - blockRatings.unpersist() - itemIdAndFactors.count() - itemFactors.unpersist() - itemInBlocks.unpersist() - } - (userIdAndFactors, itemIdAndFactors) + val shim = ALSShim.create() + shim.train( + ratings, + rank, + numUserBlocks, + numItemBlocks, + maxIter, + regParam, + implicitPrefs, + alpha, + nonnegative, + intermediateRDDStorageLevel, + finalRDDStorageLevel, + checkpointInterval, + seed + ) } /** diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/spark312/ALS.scala similarity index 60% rename from mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/recommendation/ALS.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/spark312/ALS.scala index 71195397b..aedadd437 100644 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/spark312/ALS.scala @@ -15,31 +15,25 @@ * limitations under the License. */ -package org.apache.spark.ml.recommendation +package org.apache.spark.ml.recommendation.spark312 import java.{util => ju} import java.io.IOException import java.util.Locale - import scala.collection.mutable import scala.reflect.ClassTag import scala.util.{Sorting, Try} import scala.util.hashing.byteswap64 - import com.github.fommil.netlib.BLAS.{getInstance => blas} +import com.intel.oap.mllib.recommendation.{ALSDALImpl, ALSShim} +import com.intel.oap.mllib.{Utils => DALUtils} import org.apache.hadoop.fs.Path -import org.json4s.DefaultFormats -import org.json4s.JsonDSL._ - -import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} +import org.apache.spark.{Partitioner, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.linalg.BLAS -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Utils => DALImplUtils, _} -import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.ml.recommendation.ALS.Rating +import org.apache.spark.ml.recommendation._ +import org.apache.spark.ml.util.DefaultParamsReadable import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.{DeterministicLevel, RDD} @@ -51,685 +45,6 @@ import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} import org.apache.spark.util.random.XORShiftRandom -/** - * Common params for ALS and ALSModel. - */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol - with HasBlockSize { - /** - * Param for the column name for user ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "user" - * @group param - */ - val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getUserCol: String = $(userCol) - - /** - * Param for the column name for item ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "item" - * @group param - */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getItemCol: String = $(itemCol) - - /** - * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is - * out of integer range or contains a fractional part. - */ - protected[recommendation] val checkedCast = udf { (n: Any) => - n match { - case v: Int => v // Avoid unnecessary casting - case v: Number => - val intV = v.intValue - // Checks if number within Int range and has no fractional part. - if (v.doubleValue == intV) { - intV - } else { - throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"and without fractional part for columns ${$(userCol)} and ${$(itemCol)}. " + - s"Value $n was either out of Integer range or contained a fractional part that " + - s"could not be converted.") - } - case _ => throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"for columns ${$(userCol)} and ${$(itemCol)}. Value $n was not numeric.") - } - } - - /** - * Param for strategy for dealing with unknown or new users/items at prediction time. - * This may be useful in cross-validation or production scenarios, for handling user/item ids - * the model has not seen in the training data. - * Supported values: - * - "nan": predicted value for unknown ids will be NaN. - * - "drop": rows in the input DataFrame containing unknown ids will be dropped from - * the output DataFrame containing predictions. - * Default: "nan". - * @group expertParam - */ - val coldStartStrategy = new Param[String](this, "coldStartStrategy", - "strategy for dealing with unknown or new users/items at prediction time. This may be " + - "useful in cross-validation or production scenarios, for handling user/item ids the model " + - "has not seen in the training data. Supported values: " + - s"${ALSModel.supportedColdStartStrategies.mkString(",")}.", - (s: String) => - ALSModel.supportedColdStartStrategies.contains(s.toLowerCase(Locale.ROOT))) - - /** @group expertGetParam */ - def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) - - setDefault(blockSize -> 4096) -} - -/** - * Common params for ALS. - */ -private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter with HasRegParam - with HasCheckpointInterval with HasSeed { - - /** - * Param for rank of the matrix factorization (positive). - * Default: 10 - * @group param - */ - val rank = new IntParam(this, "rank", "rank of the factorization", ParamValidators.gtEq(1)) - - /** @group getParam */ - def getRank: Int = $(rank) - - /** - * Param for number of user blocks (positive). - * Default: 10 - * @group param - */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumUserBlocks: Int = $(numUserBlocks) - - /** - * Param for number of item blocks (positive). - * Default: 10 - * @group param - */ - val numItemBlocks = new IntParam(this, "numItemBlocks", "number of item blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumItemBlocks: Int = $(numItemBlocks) - - /** - * Param to decide whether to use implicit preference. - * Default: false - * @group param - */ - val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") - - /** @group getParam */ - def getImplicitPrefs: Boolean = $(implicitPrefs) - - /** - * Param for the alpha parameter in the implicit preference formulation (nonnegative). - * Default: 1.0 - * @group param - */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", - ParamValidators.gtEq(0)) - - /** @group getParam */ - def getAlpha: Double = $(alpha) - - /** - * Param for the column name for ratings. - * Default: "rating" - * @group param - */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") - - /** @group getParam */ - def getRatingCol: String = $(ratingCol) - - /** - * Param for whether to apply nonnegativity constraints. - * Default: false - * @group param - */ - val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares") - - /** @group getParam */ - def getNonnegative: Boolean = $(nonnegative) - - /** - * Param for StorageLevel for intermediate datasets. Pass in a string representation of - * `StorageLevel`. Cannot be "NONE". - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") - - /** @group expertGetParam */ - def getIntermediateStorageLevel: String = $(intermediateStorageLevel) - - /** - * Param for StorageLevel for ALS model factors. Pass in a string representation of - * `StorageLevel`. - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val finalStorageLevel = new Param[String](this, "finalStorageLevel", - "StorageLevel for ALS model factors.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess) - - /** @group expertGetParam */ - def getFinalStorageLevel: String = $(finalStorageLevel) - - setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, - implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", - ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10, - intermediateStorageLevel -> "MEMORY_AND_DISK", finalStorageLevel -> "MEMORY_AND_DISK", - coldStartStrategy -> "nan") - - /** - * Validates and transforms the input schema. - * - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - // rating will be cast to Float - SchemaUtils.checkNumericType(schema, $(ratingCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } -} - -/** - * Model fitted by ALS. - * - * @param rank rank of the matrix factorization model - * @param userFactors a DataFrame that stores user factors in two columns: `id` and `features` - * @param itemFactors a DataFrame that stores item factors in two columns: `id` and `features` - */ -@Since("1.3.0") -class ALSModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("1.4.0") val rank: Int, - @transient val userFactors: DataFrame, - @transient val itemFactors: DataFrame) - extends Model[ALSModel] with ALSModelParams with MLWritable { - - /** @group setParam */ - @Since("1.4.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.4.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => - if (featuresA != null && featuresB != null) { - var dotProduct = 0.0f - var i = 0 - while (i < rank) { - dotProduct += featuresA(i) * featuresB(i) - i += 1 - } - dotProduct - } else { - Float.NaN - } - } - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema) - // create a new column named map(predictionCol) by running the predict UDF. - val predictions = dataset - .join(userFactors, - checkedCast(dataset($(userCol))) === userFactors("id"), "left") - .join(itemFactors, - checkedCast(dataset($(itemCol))) === itemFactors("id"), "left") - .select(dataset("*"), - predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) - getColdStartStrategy match { - case ALSModel.Drop => - predictions.na.drop("all", Seq($(predictionCol))) - case ALSModel.NaN => - predictions - } - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALSModel = { - val copied = new ALSModel(uid, rank, userFactors, itemFactors) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new ALSModel.ALSModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"ALSModel: uid=$uid, rank=$rank" - } - - /** - * Returns top `numItems` items recommended for each user, for all users. - * @param numItems max number of recommendations for each user - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numItems` items recommended for each user id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of user ids. The column name must match `userCol`. - * @param numItems max number of recommendations for each user. - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item, for all items. - * @param numUsers max number of recommendations for each item - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of item ids. The column name must match `itemCol`. - * @param numUsers max number of recommendations for each item. - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns a subset of a factor DataFrame limited to only those unique ids contained - * in the input dataset. - * @param dataset input Dataset containing id column to user to filter factors. - * @param factors factor DataFrame to filter. - * @param column column name containing the ids in the input dataset. - * @return DataFrame containing factors only for those ids present in both the input dataset and - * the factor DataFrame. - */ - private def getSourceFactorSubset( - dataset: Dataset[_], - factors: DataFrame, - column: String): DataFrame = { - factors - .join(dataset.select(column), factors("id") === dataset(column), joinType = "left_semi") - .select(factors("id"), factors("features")) - } - - /** - * Makes recommendations for all users (or items). - * - * Note: the previous approach used for computing top-k recommendations - * used a cross-join followed by predicting a score for each row of the joined dataset. - * However, this results in exploding the size of intermediate data. While Spark SQL makes it - * relatively efficient, the approach implemented here is significantly more efficient. - * - * This approach groups factors into blocks and computes the top-k elements per block, - * using dot product and an efficient [[BoundedPriorityQueue]] (instead of gemm). - * It then computes the global top-k by aggregating the per block top-k elements with - * a [[TopByKeyAggregator]]. This significantly reduces the size of intermediate and shuffle data. - * This is the DataFrame equivalent to the approach used in - * [[org.apache.spark.mllib.recommendation.MatrixFactorizationModel]]. - * - * @param srcFactors src factors for which to generate recommendations - * @param dstFactors dst factors used to make recommendations - * @param srcOutputColumn name of the column for the source ID in the output DataFrame - * @param dstOutputColumn name of the column for the destination ID in the output DataFrame - * @param num max number of recommendations for each record - * @return a DataFrame of (srcOutputColumn: Int, recommendations), where recommendations are - * stored as an array of (dstOutputColumn: Int, rating: Float) Rows. - */ - private def recommendForAll( - srcFactors: DataFrame, - dstFactors: DataFrame, - srcOutputColumn: String, - dstOutputColumn: String, - num: Int, - blockSize: Int): DataFrame = { - import srcFactors.sparkSession.implicits._ - - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) - val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) - .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] - .flatMap { case (srcIter, dstIter) => - val m = srcIter.size - val n = math.min(dstIter.size, num) - val output = new Array[(Int, Int, Float)](m * n) - var i = 0 - val pq = new BoundedPriorityQueue[(Int, Float)](num)(Ordering.by(_._2)) - srcIter.foreach { case (srcId, srcFactor) => - dstIter.foreach { case (dstId, dstFactor) => - // We use F2jBLAS which is faster than a call to native BLAS for vector dot product - val score = BLAS.f2jBLAS.sdot(rank, srcFactor, 1, dstFactor, 1) - pq += dstId -> score - } - pq.foreach { case (dstId, score) => - output(i) = (srcId, dstId, score) - i += 1 - } - pq.clear() - } - output.toSeq - } - // We'll force the IDs to be Int. Unfortunately this converts IDs to Int in the output. - val topKAggregator = new TopByKeyAggregator[Int, Int, Float](num, Ordering.by(_._2)) - val recs = ratings.as[(Int, Int, Float)].groupByKey(_._1).agg(topKAggregator.toColumn) - .toDF("id", "recommendations") - - val arrayType = ArrayType( - new StructType() - .add(dstOutputColumn, IntegerType) - .add("rating", FloatType) - ) - recs.select($"id".as(srcOutputColumn), $"recommendations".cast(arrayType)) - } - - /** - * Blockifies factors to improve the efficiency of cross join - */ - private def blockify( - factors: Dataset[(Int, Array[Float])], - blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { - import factors.sparkSession.implicits._ - factors.mapPartitions(_.grouped(blockSize)) - } - -} - -@Since("1.6.0") -object ALSModel extends MLReadable[ALSModel] { - - private val NaN = "nan" - private val Drop = "drop" - private[recommendation] final val supportedColdStartStrategies = Array(NaN, Drop) - - @Since("1.6.0") - override def read: MLReader[ALSModel] = new ALSModelReader - - @Since("1.6.0") - override def load(path: String): ALSModel = super.load(path) - - private[ALSModel] class ALSModelWriter(instance: ALSModel) extends MLWriter { - - override protected def saveImpl(path: String): Unit = { - val extraMetadata = "rank" -> instance.rank - DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) - val userPath = new Path(path, "userFactors").toString - instance.userFactors.write.format("parquet").save(userPath) - val itemPath = new Path(path, "itemFactors").toString - instance.itemFactors.write.format("parquet").save(itemPath) - } - } - - private class ALSModelReader extends MLReader[ALSModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[ALSModel].getName - - override def load(path: String): ALSModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - implicit val format = DefaultFormats - val rank = (metadata.metadata \ "rank").extract[Int] - val userPath = new Path(path, "userFactors").toString - val userFactors = sparkSession.read.format("parquet").load(userPath) - val itemPath = new Path(path, "itemFactors").toString - val itemFactors = sparkSession.read.format("parquet").load(itemPath) - - val model = new ALSModel(metadata.uid, rank, userFactors, itemFactors) - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Alternating Least Squares (ALS) matrix factorization. - * - * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices, - * `X` and `Y`, i.e. `X * Yt = R`. Typically these approximations are called 'factor' matrices. - * The general approach is iterative. During each iteration, one of the factor matrices is held - * constant, while the other is solved for using least squares. The newly-solved factor matrix is - * then held constant while solving for the other factor matrix. - * - * This is a blocked implementation of the ALS factorization algorithm that groups the two sets - * of factors (referred to as "users" and "products") into blocks and reduces communication by only - * sending one copy of each user vector to each product block on each iteration, and only for the - * product blocks that need that user's feature vector. This is achieved by pre-computing some - * information about the ratings matrix to determine the "out-links" of each user (which blocks of - * products it will contribute to) and "in-link" information for each product (which of the feature - * vectors it receives from each user block it will depend on). This allows us to send only an - * array of feature vectors between each user block and product block, and have the product block - * find the users' ratings and update the products based on these messages. - * - * For implicit preference data, the algorithm used is based on - * "Collaborative Filtering for Implicit Feedback Datasets", available at - * https://doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. - * - * Essentially instead of finding the low-rank approximations to the rating matrix `R`, - * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if - * r is greater than 0 and 0 if r is less than or equal to 0. The ratings then act as 'confidence' - * values related to strength of indicated user - * preferences rather than explicit ratings given to items. - * - * Note: the input rating dataset to the ALS implementation should be deterministic. - * Nondeterministic data can cause failure during fitting ALS model. - * For example, an order-sensitive operation like sampling after a repartition makes dataset - * output nondeterministic, like `dataset.repartition(2).sample(false, 0.5, 1618)`. - * Checkpointing sampled dataset or adding a sort before sampling can help make the dataset - * deterministic. - */ -@Since("1.3.0") -class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams - with DefaultParamsWritable { - - import org.apache.spark.ml.recommendation.ALS.Rating - - @Since("1.4.0") - def this() = this(Identifiable.randomUID("als")) - - /** @group setParam */ - @Since("1.3.0") - def setRank(value: Int): this.type = set(rank, value) - - /** @group setParam */ - @Since("1.3.0") - def setNumUserBlocks(value: Int): this.type = set(numUserBlocks, value) - - /** @group setParam */ - @Since("1.3.0") - def setNumItemBlocks(value: Int): this.type = set(numItemBlocks, value) - - /** @group setParam */ - @Since("1.3.0") - def setImplicitPrefs(value: Boolean): this.type = set(implicitPrefs, value) - - /** @group setParam */ - @Since("1.3.0") - def setAlpha(value: Double): this.type = set(alpha, value) - - /** @group setParam */ - @Since("1.3.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setRatingCol(value: String): this.type = set(ratingCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - - /** @group setParam */ - @Since("1.3.0") - def setNonnegative(value: Boolean): this.type = set(nonnegative, value) - - /** @group setParam */ - @Since("1.4.0") - def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) - - /** @group setParam */ - @Since("1.3.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** @group expertSetParam */ - @Since("2.0.0") - def setIntermediateStorageLevel(value: String): this.type = set(intermediateStorageLevel, value) - - /** @group expertSetParam */ - @Since("2.0.0") - def setFinalStorageLevel(value: String): this.type = set(finalStorageLevel, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - /** - * Sets both numUserBlocks and numItemBlocks to the specific value. - * - * @group setParam - */ - @Since("1.3.0") - def setNumBlocks(value: Int): this.type = { - setNumUserBlocks(value) - setNumItemBlocks(value) - this - } - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): ALSModel = instrumented { instr => - transformSchema(dataset.schema) - import dataset.sparkSession.implicits._ - - val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) - val ratings = dataset - .select(checkedCast(col($(userCol))), checkedCast(col($(itemCol))), r) - .rdd - .map { row => - Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) - } - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, - itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, - seed, intermediateStorageLevel, finalStorageLevel, blockSize) - - val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), - numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), - maxIter = $(maxIter), regParam = $(regParam), implicitPrefs = $(implicitPrefs), - alpha = $(alpha), nonnegative = $(nonnegative), - intermediateRDDStorageLevel = StorageLevel.fromString($(intermediateStorageLevel)), - finalRDDStorageLevel = StorageLevel.fromString($(finalStorageLevel)), - checkpointInterval = $(checkpointInterval), seed = $(seed)) - val userDF = userFactors.toDF("id", "features") - val itemDF = itemFactors.toDF("id", "features") - val model = new ALSModel(uid, $(rank), userDF, itemDF).setBlockSize($(blockSize)) - .setParent(this) - copyValues(model) - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALS = defaultCopy(extra) -} - - /** * An implementation of ALS that supports generic ID types, specialized for Int and Long. This is * exposed as a developer API for users who do need other ID types. But it is not recommended @@ -737,12 +52,12 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] * users and items must have the same type. The number of distinct users/items should be smaller * than 2 billion. */ -object ALS extends DefaultParamsReadable[ALS] with Logging { +class ALS extends DefaultParamsReadable[ALS] with Logging with ALSShim { /** * Rating class for better code readability. */ - case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) +// case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) @Since("1.6.0") override def load(path: String): ALS = super.load(path) @@ -919,10 +234,10 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { seed: Long = 0L)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - val isPlatformSupported = DALImplUtils.checkClusterPlatformCompatibility(ratings.sparkContext) + val isPlatformSupported = DALUtils.checkClusterPlatformCompatibility(ratings.sparkContext) val (userIdAndFactors, itemIdAndFactors) = - if (implicitPrefs && DALImplUtils.isOAPEnabled() && isPlatformSupported) { + if (implicitPrefs && DALUtils.isOAPEnabled() && isPlatformSupported) { new ALSDALImpl(ratings, rank, maxIter, regParam, alpha, seed).train() } else { trainMLlib(ratings, rank, numUserBlocks, numItemBlocks, maxIter, regParam, implicitPrefs, @@ -933,16 +248,6 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { (userIdAndFactors, itemIdAndFactors) } - private def trainDAL[ID: ClassTag]( - ratings: RDD[Rating[ID]], - rank: Int, - maxIter: Int, - regParam: Double, - alpha: Double, - seed: Long): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - null - } - /** * Implementation of the ALS algorithm. * @@ -968,7 +273,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * "block" as referring to a subset of an RDD containing the ratings rather than a contiguous * submatrix of the ratings matrix. */ - private def trainMLlib[ID: ClassTag]( // scalastyle:ignore + def trainMLlib[ID: ClassTag]( // scalastyle:ignore ratings: RDD[Rating[ID]], rank: Int = 10, numUserBlocks: Int = 10, @@ -1043,7 +348,6 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) // TODO: Generalize PeriodicGraphCheckpointer and use it here. - val deps = itemFactors.dependencies if (shouldCheckpoint(iter)) { itemFactors.checkpoint() // itemFactors gets materialized in computeFactors } @@ -1051,7 +355,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) if (shouldCheckpoint(iter)) { - ALS.cleanShuffleDependencies(sc, deps) + itemFactors.cleanShuffleDependencies() deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -1064,10 +368,9 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { userLocalIndexEncoder, solver = solver) if (shouldCheckpoint(iter)) { itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) - val deps = itemFactors.dependencies itemFactors.checkpoint() itemFactors.count() // checkpoint item factors and cut lineage - ALS.cleanShuffleDependencies(sc, deps) + itemFactors.cleanShuffleDependencies() deletePreviousCheckpointFile() previousCachedItemFactors.foreach(_.unpersist()) @@ -1083,7 +386,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { .join(userFactors) .mapPartitions({ items => items.flatMap { case (_, (ids, factors)) => - ids.view.zip(factors) + ids.iterator.zip(factors.iterator) } // Preserve the partitioning because IDs are consistent with the partitioners in userInBlocks // and userFactors. @@ -1095,7 +398,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { .join(itemFactors) .mapPartitions({ items => items.flatMap { case (_, (ids, factors)) => - ids.view.zip(factors) + ids.iterator.zip(factors.iterator) } }, preservesPartitioning = true) .setName("itemFactors") @@ -1410,7 +713,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { Iterator.empty } } ++ { - builders.view.zipWithIndex.filter(_._1.size > 0).map { case (block, idx) => + builders.iterator.zipWithIndex.filter(_._1.size > 0).map { case (block, idx) => val srcBlockId = idx % srcPart.numPartitions val dstBlockId = idx / srcPart.numPartitions ((srcBlockId, dstBlockId), block.build()) @@ -1729,7 +1032,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { val YtY = if (implicitPrefs) Some(computeYtY(srcFactorBlocks, rank)) else None val srcOut = srcOutBlocks.join(srcFactorBlocks).flatMap { case (srcBlockId, (srcOutBlock, srcFactors)) => - srcOutBlock.view.zipWithIndex.map { case (activeIndices, dstBlockId) => + srcOutBlock.iterator.zipWithIndex.map { case (activeIndices, dstBlockId) => (dstBlockId, (srcBlockId, activeIndices.map(idx => srcFactors(idx)))) } } @@ -1855,31 +1158,4 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * satisfies this requirement, we simply use a type alias here. */ private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner - - /** - * Private function to clean up all of the shuffles files from the dependencies and their parents. - */ - private[spark] def cleanShuffleDependencies[T]( - sc: SparkContext, - deps: Seq[Dependency[_]], - blocking: Boolean = false): Unit = { - // If there is no reference tracking we skip clean up. - sc.cleaner.foreach { cleaner => - /** - * Clean the shuffles & all of its parents. - */ - def cleanEagerly(dep: Dependency[_]): Unit = { - if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { - val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId - cleaner.doCleanupShuffle(shuffleId, blocking) - } - val rdd = dep.rdd - val rddDeps = rdd.dependencies - if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { - rddDeps.foreach(cleanEagerly) - } - } - deps.foreach(cleanEagerly) - } - } } diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/spark320/ALS.scala similarity index 62% rename from mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/spark320/ALS.scala index 71195397b..c0d408728 100644 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/spark320/ALS.scala @@ -15,720 +15,31 @@ * limitations under the License. */ -package org.apache.spark.ml.recommendation - -import java.{util => ju} -import java.io.IOException -import java.util.Locale - -import scala.collection.mutable -import scala.reflect.ClassTag -import scala.util.{Sorting, Try} -import scala.util.hashing.byteswap64 +package org.apache.spark.ml.recommendation.spark320 import com.github.fommil.netlib.BLAS.{getInstance => blas} +import com.intel.oap.mllib.recommendation.{ALSDALImpl, ALSShim} +import com.intel.oap.mllib.{Utils => DALUtils} import org.apache.hadoop.fs.Path -import org.json4s.DefaultFormats -import org.json4s.JsonDSL._ - -import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} -import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.linalg.BLAS -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Utils => DALImplUtils, _} -import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.ml.recommendation.ALS.Rating +import org.apache.spark.ml.recommendation._ +import org.apache.spark.ml.util.DefaultParamsReadable import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.{DeterministicLevel, RDD} -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.Utils import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} -/** - * Common params for ALS and ALSModel. - */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol - with HasBlockSize { - /** - * Param for the column name for user ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "user" - * @group param - */ - val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getUserCol: String = $(userCol) - - /** - * Param for the column name for item ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "item" - * @group param - */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getItemCol: String = $(itemCol) - - /** - * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is - * out of integer range or contains a fractional part. - */ - protected[recommendation] val checkedCast = udf { (n: Any) => - n match { - case v: Int => v // Avoid unnecessary casting - case v: Number => - val intV = v.intValue - // Checks if number within Int range and has no fractional part. - if (v.doubleValue == intV) { - intV - } else { - throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"and without fractional part for columns ${$(userCol)} and ${$(itemCol)}. " + - s"Value $n was either out of Integer range or contained a fractional part that " + - s"could not be converted.") - } - case _ => throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"for columns ${$(userCol)} and ${$(itemCol)}. Value $n was not numeric.") - } - } - - /** - * Param for strategy for dealing with unknown or new users/items at prediction time. - * This may be useful in cross-validation or production scenarios, for handling user/item ids - * the model has not seen in the training data. - * Supported values: - * - "nan": predicted value for unknown ids will be NaN. - * - "drop": rows in the input DataFrame containing unknown ids will be dropped from - * the output DataFrame containing predictions. - * Default: "nan". - * @group expertParam - */ - val coldStartStrategy = new Param[String](this, "coldStartStrategy", - "strategy for dealing with unknown or new users/items at prediction time. This may be " + - "useful in cross-validation or production scenarios, for handling user/item ids the model " + - "has not seen in the training data. Supported values: " + - s"${ALSModel.supportedColdStartStrategies.mkString(",")}.", - (s: String) => - ALSModel.supportedColdStartStrategies.contains(s.toLowerCase(Locale.ROOT))) - - /** @group expertGetParam */ - def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) - - setDefault(blockSize -> 4096) -} - -/** - * Common params for ALS. - */ -private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter with HasRegParam - with HasCheckpointInterval with HasSeed { - - /** - * Param for rank of the matrix factorization (positive). - * Default: 10 - * @group param - */ - val rank = new IntParam(this, "rank", "rank of the factorization", ParamValidators.gtEq(1)) - - /** @group getParam */ - def getRank: Int = $(rank) - - /** - * Param for number of user blocks (positive). - * Default: 10 - * @group param - */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumUserBlocks: Int = $(numUserBlocks) - - /** - * Param for number of item blocks (positive). - * Default: 10 - * @group param - */ - val numItemBlocks = new IntParam(this, "numItemBlocks", "number of item blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumItemBlocks: Int = $(numItemBlocks) - - /** - * Param to decide whether to use implicit preference. - * Default: false - * @group param - */ - val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") - - /** @group getParam */ - def getImplicitPrefs: Boolean = $(implicitPrefs) - - /** - * Param for the alpha parameter in the implicit preference formulation (nonnegative). - * Default: 1.0 - * @group param - */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", - ParamValidators.gtEq(0)) - - /** @group getParam */ - def getAlpha: Double = $(alpha) - - /** - * Param for the column name for ratings. - * Default: "rating" - * @group param - */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") - - /** @group getParam */ - def getRatingCol: String = $(ratingCol) - - /** - * Param for whether to apply nonnegativity constraints. - * Default: false - * @group param - */ - val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares") - - /** @group getParam */ - def getNonnegative: Boolean = $(nonnegative) - - /** - * Param for StorageLevel for intermediate datasets. Pass in a string representation of - * `StorageLevel`. Cannot be "NONE". - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") - - /** @group expertGetParam */ - def getIntermediateStorageLevel: String = $(intermediateStorageLevel) - - /** - * Param for StorageLevel for ALS model factors. Pass in a string representation of - * `StorageLevel`. - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val finalStorageLevel = new Param[String](this, "finalStorageLevel", - "StorageLevel for ALS model factors.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess) - - /** @group expertGetParam */ - def getFinalStorageLevel: String = $(finalStorageLevel) - - setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, - implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", - ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10, - intermediateStorageLevel -> "MEMORY_AND_DISK", finalStorageLevel -> "MEMORY_AND_DISK", - coldStartStrategy -> "nan") - - /** - * Validates and transforms the input schema. - * - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - // rating will be cast to Float - SchemaUtils.checkNumericType(schema, $(ratingCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } -} - -/** - * Model fitted by ALS. - * - * @param rank rank of the matrix factorization model - * @param userFactors a DataFrame that stores user factors in two columns: `id` and `features` - * @param itemFactors a DataFrame that stores item factors in two columns: `id` and `features` - */ -@Since("1.3.0") -class ALSModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("1.4.0") val rank: Int, - @transient val userFactors: DataFrame, - @transient val itemFactors: DataFrame) - extends Model[ALSModel] with ALSModelParams with MLWritable { - - /** @group setParam */ - @Since("1.4.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.4.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => - if (featuresA != null && featuresB != null) { - var dotProduct = 0.0f - var i = 0 - while (i < rank) { - dotProduct += featuresA(i) * featuresB(i) - i += 1 - } - dotProduct - } else { - Float.NaN - } - } - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema) - // create a new column named map(predictionCol) by running the predict UDF. - val predictions = dataset - .join(userFactors, - checkedCast(dataset($(userCol))) === userFactors("id"), "left") - .join(itemFactors, - checkedCast(dataset($(itemCol))) === itemFactors("id"), "left") - .select(dataset("*"), - predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) - getColdStartStrategy match { - case ALSModel.Drop => - predictions.na.drop("all", Seq($(predictionCol))) - case ALSModel.NaN => - predictions - } - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALSModel = { - val copied = new ALSModel(uid, rank, userFactors, itemFactors) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new ALSModel.ALSModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"ALSModel: uid=$uid, rank=$rank" - } - - /** - * Returns top `numItems` items recommended for each user, for all users. - * @param numItems max number of recommendations for each user - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numItems` items recommended for each user id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of user ids. The column name must match `userCol`. - * @param numItems max number of recommendations for each user. - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item, for all items. - * @param numUsers max number of recommendations for each item - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of item ids. The column name must match `itemCol`. - * @param numUsers max number of recommendations for each item. - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns a subset of a factor DataFrame limited to only those unique ids contained - * in the input dataset. - * @param dataset input Dataset containing id column to user to filter factors. - * @param factors factor DataFrame to filter. - * @param column column name containing the ids in the input dataset. - * @return DataFrame containing factors only for those ids present in both the input dataset and - * the factor DataFrame. - */ - private def getSourceFactorSubset( - dataset: Dataset[_], - factors: DataFrame, - column: String): DataFrame = { - factors - .join(dataset.select(column), factors("id") === dataset(column), joinType = "left_semi") - .select(factors("id"), factors("features")) - } - - /** - * Makes recommendations for all users (or items). - * - * Note: the previous approach used for computing top-k recommendations - * used a cross-join followed by predicting a score for each row of the joined dataset. - * However, this results in exploding the size of intermediate data. While Spark SQL makes it - * relatively efficient, the approach implemented here is significantly more efficient. - * - * This approach groups factors into blocks and computes the top-k elements per block, - * using dot product and an efficient [[BoundedPriorityQueue]] (instead of gemm). - * It then computes the global top-k by aggregating the per block top-k elements with - * a [[TopByKeyAggregator]]. This significantly reduces the size of intermediate and shuffle data. - * This is the DataFrame equivalent to the approach used in - * [[org.apache.spark.mllib.recommendation.MatrixFactorizationModel]]. - * - * @param srcFactors src factors for which to generate recommendations - * @param dstFactors dst factors used to make recommendations - * @param srcOutputColumn name of the column for the source ID in the output DataFrame - * @param dstOutputColumn name of the column for the destination ID in the output DataFrame - * @param num max number of recommendations for each record - * @return a DataFrame of (srcOutputColumn: Int, recommendations), where recommendations are - * stored as an array of (dstOutputColumn: Int, rating: Float) Rows. - */ - private def recommendForAll( - srcFactors: DataFrame, - dstFactors: DataFrame, - srcOutputColumn: String, - dstOutputColumn: String, - num: Int, - blockSize: Int): DataFrame = { - import srcFactors.sparkSession.implicits._ - - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) - val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) - .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] - .flatMap { case (srcIter, dstIter) => - val m = srcIter.size - val n = math.min(dstIter.size, num) - val output = new Array[(Int, Int, Float)](m * n) - var i = 0 - val pq = new BoundedPriorityQueue[(Int, Float)](num)(Ordering.by(_._2)) - srcIter.foreach { case (srcId, srcFactor) => - dstIter.foreach { case (dstId, dstFactor) => - // We use F2jBLAS which is faster than a call to native BLAS for vector dot product - val score = BLAS.f2jBLAS.sdot(rank, srcFactor, 1, dstFactor, 1) - pq += dstId -> score - } - pq.foreach { case (dstId, score) => - output(i) = (srcId, dstId, score) - i += 1 - } - pq.clear() - } - output.toSeq - } - // We'll force the IDs to be Int. Unfortunately this converts IDs to Int in the output. - val topKAggregator = new TopByKeyAggregator[Int, Int, Float](num, Ordering.by(_._2)) - val recs = ratings.as[(Int, Int, Float)].groupByKey(_._1).agg(topKAggregator.toColumn) - .toDF("id", "recommendations") - - val arrayType = ArrayType( - new StructType() - .add(dstOutputColumn, IntegerType) - .add("rating", FloatType) - ) - recs.select($"id".as(srcOutputColumn), $"recommendations".cast(arrayType)) - } - - /** - * Blockifies factors to improve the efficiency of cross join - */ - private def blockify( - factors: Dataset[(Int, Array[Float])], - blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { - import factors.sparkSession.implicits._ - factors.mapPartitions(_.grouped(blockSize)) - } - -} - -@Since("1.6.0") -object ALSModel extends MLReadable[ALSModel] { - - private val NaN = "nan" - private val Drop = "drop" - private[recommendation] final val supportedColdStartStrategies = Array(NaN, Drop) - - @Since("1.6.0") - override def read: MLReader[ALSModel] = new ALSModelReader - - @Since("1.6.0") - override def load(path: String): ALSModel = super.load(path) - - private[ALSModel] class ALSModelWriter(instance: ALSModel) extends MLWriter { - - override protected def saveImpl(path: String): Unit = { - val extraMetadata = "rank" -> instance.rank - DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) - val userPath = new Path(path, "userFactors").toString - instance.userFactors.write.format("parquet").save(userPath) - val itemPath = new Path(path, "itemFactors").toString - instance.itemFactors.write.format("parquet").save(itemPath) - } - } - - private class ALSModelReader extends MLReader[ALSModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[ALSModel].getName - - override def load(path: String): ALSModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - implicit val format = DefaultFormats - val rank = (metadata.metadata \ "rank").extract[Int] - val userPath = new Path(path, "userFactors").toString - val userFactors = sparkSession.read.format("parquet").load(userPath) - val itemPath = new Path(path, "itemFactors").toString - val itemFactors = sparkSession.read.format("parquet").load(itemPath) - - val model = new ALSModel(metadata.uid, rank, userFactors, itemFactors) - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Alternating Least Squares (ALS) matrix factorization. - * - * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices, - * `X` and `Y`, i.e. `X * Yt = R`. Typically these approximations are called 'factor' matrices. - * The general approach is iterative. During each iteration, one of the factor matrices is held - * constant, while the other is solved for using least squares. The newly-solved factor matrix is - * then held constant while solving for the other factor matrix. - * - * This is a blocked implementation of the ALS factorization algorithm that groups the two sets - * of factors (referred to as "users" and "products") into blocks and reduces communication by only - * sending one copy of each user vector to each product block on each iteration, and only for the - * product blocks that need that user's feature vector. This is achieved by pre-computing some - * information about the ratings matrix to determine the "out-links" of each user (which blocks of - * products it will contribute to) and "in-link" information for each product (which of the feature - * vectors it receives from each user block it will depend on). This allows us to send only an - * array of feature vectors between each user block and product block, and have the product block - * find the users' ratings and update the products based on these messages. - * - * For implicit preference data, the algorithm used is based on - * "Collaborative Filtering for Implicit Feedback Datasets", available at - * https://doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. - * - * Essentially instead of finding the low-rank approximations to the rating matrix `R`, - * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if - * r is greater than 0 and 0 if r is less than or equal to 0. The ratings then act as 'confidence' - * values related to strength of indicated user - * preferences rather than explicit ratings given to items. - * - * Note: the input rating dataset to the ALS implementation should be deterministic. - * Nondeterministic data can cause failure during fitting ALS model. - * For example, an order-sensitive operation like sampling after a repartition makes dataset - * output nondeterministic, like `dataset.repartition(2).sample(false, 0.5, 1618)`. - * Checkpointing sampled dataset or adding a sort before sampling can help make the dataset - * deterministic. - */ -@Since("1.3.0") -class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams - with DefaultParamsWritable { - - import org.apache.spark.ml.recommendation.ALS.Rating - - @Since("1.4.0") - def this() = this(Identifiable.randomUID("als")) - - /** @group setParam */ - @Since("1.3.0") - def setRank(value: Int): this.type = set(rank, value) - - /** @group setParam */ - @Since("1.3.0") - def setNumUserBlocks(value: Int): this.type = set(numUserBlocks, value) - - /** @group setParam */ - @Since("1.3.0") - def setNumItemBlocks(value: Int): this.type = set(numItemBlocks, value) - - /** @group setParam */ - @Since("1.3.0") - def setImplicitPrefs(value: Boolean): this.type = set(implicitPrefs, value) - - /** @group setParam */ - @Since("1.3.0") - def setAlpha(value: Double): this.type = set(alpha, value) - - /** @group setParam */ - @Since("1.3.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setRatingCol(value: String): this.type = set(ratingCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - - /** @group setParam */ - @Since("1.3.0") - def setNonnegative(value: Boolean): this.type = set(nonnegative, value) - - /** @group setParam */ - @Since("1.4.0") - def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) - - /** @group setParam */ - @Since("1.3.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** @group expertSetParam */ - @Since("2.0.0") - def setIntermediateStorageLevel(value: String): this.type = set(intermediateStorageLevel, value) - - /** @group expertSetParam */ - @Since("2.0.0") - def setFinalStorageLevel(value: String): this.type = set(finalStorageLevel, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - /** - * Sets both numUserBlocks and numItemBlocks to the specific value. - * - * @group setParam - */ - @Since("1.3.0") - def setNumBlocks(value: Int): this.type = { - setNumUserBlocks(value) - setNumItemBlocks(value) - this - } - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): ALSModel = instrumented { instr => - transformSchema(dataset.schema) - import dataset.sparkSession.implicits._ - - val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) - val ratings = dataset - .select(checkedCast(col($(userCol))), checkedCast(col($(itemCol))), r) - .rdd - .map { row => - Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) - } - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, - itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, - seed, intermediateStorageLevel, finalStorageLevel, blockSize) - - val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), - numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), - maxIter = $(maxIter), regParam = $(regParam), implicitPrefs = $(implicitPrefs), - alpha = $(alpha), nonnegative = $(nonnegative), - intermediateRDDStorageLevel = StorageLevel.fromString($(intermediateStorageLevel)), - finalRDDStorageLevel = StorageLevel.fromString($(finalStorageLevel)), - checkpointInterval = $(checkpointInterval), seed = $(seed)) - val userDF = userFactors.toDF("id", "features") - val itemDF = itemFactors.toDF("id", "features") - val model = new ALSModel(uid, $(rank), userDF, itemDF).setBlockSize($(blockSize)) - .setParent(this) - copyValues(model) - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALS = defaultCopy(extra) -} - +import java.io.IOException +import java.{util => ju} +import scala.collection.mutable +import scala.reflect.ClassTag +import scala.util.Sorting +import scala.util.hashing.byteswap64 /** * An implementation of ALS that supports generic ID types, specialized for Int and Long. This is @@ -737,15 +48,15 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] * users and items must have the same type. The number of distinct users/items should be smaller * than 2 billion. */ -object ALS extends DefaultParamsReadable[ALS] with Logging { +class ALS extends DefaultParamsReadable[ALS] with Logging with ALSShim { /** * Rating class for better code readability. */ - case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) + // case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) - @Since("1.6.0") - override def load(path: String): ALS = super.load(path) + // @Since("1.6.0") + // override def load(path: String): ALS = super.load(path) /** Trait for least squares solvers applied to the normal equation. */ private[recommendation] trait LeastSquaresNESolver extends Serializable { @@ -919,10 +230,10 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { seed: Long = 0L)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - val isPlatformSupported = DALImplUtils.checkClusterPlatformCompatibility(ratings.sparkContext) + val isPlatformSupported = DALUtils.checkClusterPlatformCompatibility(ratings.sparkContext) val (userIdAndFactors, itemIdAndFactors) = - if (implicitPrefs && DALImplUtils.isOAPEnabled() && isPlatformSupported) { + if (implicitPrefs && DALUtils.isOAPEnabled() && isPlatformSupported) { new ALSDALImpl(ratings, rank, maxIter, regParam, alpha, seed).train() } else { trainMLlib(ratings, rank, numUserBlocks, numItemBlocks, maxIter, regParam, implicitPrefs, @@ -933,16 +244,6 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { (userIdAndFactors, itemIdAndFactors) } - private def trainDAL[ID: ClassTag]( - ratings: RDD[Rating[ID]], - rank: Int, - maxIter: Int, - regParam: Double, - alpha: Double, - seed: Long): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - null - } - /** * Implementation of the ALS algorithm. * diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala new file mode 100644 index 000000000..a45ff6051 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.regression + +import com.intel.oap.mllib.regression.LinearRegressionShim +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.optim.WeightedLeastSquares +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util._ +import org.apache.spark.sql.Dataset + +/** + * Linear regression. + * + * The learning objective is to minimize the specified loss function, with regularization. + * This supports two kinds of loss: + * - squaredError (a.k.a squared loss) + * - huber (a hybrid of squared error for relatively small errors and absolute error for + * relatively large ones, and we estimate the scale parameter from training data) + * + * This supports multiple types of regularization: + * - none (a.k.a. ordinary least squares) + * - L2 (ridge regression) + * - L1 (Lasso) + * - L2 + L1 (elastic net) + * + * The squared error objective function is: + * + *
+ * $$ + * \begin{align} + * \min_{w}\frac{1}{2n}{\sum_{i=1}^n(X_{i}w - y_{i})^{2} + + * \lambda\left[\frac{1-\alpha}{2}{||w||_{2}}^{2} + \alpha{||w||_{1}}\right]} + * \end{align} + * $$ + *
+ * + * The huber objective function is: + * + *
+ * $$ + * \begin{align} + * \min_{w, \sigma}\frac{1}{2n}{\sum_{i=1}^n\left(\sigma + + * H_m\left(\frac{X_{i}w - y_{i}}{\sigma}\right)\sigma\right) + \frac{1}{2}\lambda {||w||_2}^2} + * \end{align} + * $$ + *
+ * + * where + * + *
+ * $$ + * \begin{align} + * H_m(z) = \begin{cases} + * z^2, & \text {if } |z| < \epsilon, \\ + * 2\epsilon|z| - \epsilon^2, & \text{otherwise} + * \end{cases} + * \end{align} + * $$ + *
+ * + * Since 3.1.0, it supports stacking instances into blocks and using GEMV for + * better performance. + * The block size will be 1.0 MB, if param maxBlockSizeInMB is set 0.0 by default. + * + * Note: Fitting with huber loss only supports none and L2 regularization. + */ +@Since("1.3.0") +class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) + extends Regressor[Vector, LinearRegression, LinearRegressionModel] + with LinearRegressionParams with DefaultParamsWritable with Logging { + + @Since("1.4.0") + def this() = this(Identifiable.randomUID("linReg")) + + /** + * Set the regularization parameter. + * Default is 0.0. + * + * @group setParam + */ + @Since("1.3.0") + def setRegParam(value: Double): this.type = set(regParam, value) + + /** + * Set if we should fit the intercept. + * Default is true. + * + * @group setParam + */ + @Since("1.5.0") + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + + /** + * Whether to standardize the training features before fitting the model. + * The coefficients of models will be always returned on the original scale, + * so it will be transparent for users. + * Default is true. + * + * @note With/without standardization, the models should be always converged + * to the same solution when no regularization is applied. In R's GLMNET package, + * the default behavior is true as well. + * + * @group setParam + */ + @Since("1.5.0") + def setStandardization(value: Boolean): this.type = set(standardization, value) + + /** + * Set the ElasticNet mixing parameter. + * For alpha = 0, the penalty is an L2 penalty. + * For alpha = 1, it is an L1 penalty. + * For alpha in (0,1), the penalty is a combination of L1 and L2. + * Default is 0.0 which is an L2 penalty. + * + * Note: Fitting with huber loss only supports None and L2 regularization, + * so throws exception if this param is non-zero value. + * + * @group setParam + */ + @Since("1.4.0") + def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) + + /** + * Set the maximum number of iterations. + * Default is 100. + * + * @group setParam + */ + @Since("1.3.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** + * Set the convergence tolerance of iterations. + * Smaller value will lead to higher accuracy with the cost of more iterations. + * Default is 1E-6. + * + * @group setParam + */ + @Since("1.4.0") + def setTol(value: Double): this.type = set(tol, value) + + /** + * Whether to over-/under-sample training instances according to the given weights in weightCol. + * If not set or empty, all instances are treated equally (weight 1.0). + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("1.6.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + /** + * Set the solver algorithm used for optimization. + * In case of linear regression, this can be "l-bfgs", "normal" and "auto". + * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton + * optimization method. + * - "normal" denotes using Normal Equation as an analytical solution to the linear regression + * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. + * - "auto" (default) means that the solver algorithm is selected automatically. + * The Normal Equations solver will be used when possible, but this will automatically fall + * back to iterative optimization methods when needed. + * + * Note: Fitting with huber loss doesn't support normal solver, + * so throws exception if this param was set with "normal". + * @group setParam + */ + @Since("1.6.0") + def setSolver(value: String): this.type = set(solver, value) + + /** + * Suggested depth for treeAggregate (greater than or equal to 2). + * If the dimensions of features or the number of partitions are large, + * this param could be adjusted to a larger size. + * Default is 2. + * + * @group expertSetParam + */ + @Since("2.1.0") + def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) + + /** + * Sets the value of param [[loss]]. + * Default is "squaredError". + * + * @group setParam + */ + @Since("2.3.0") + def setLoss(value: String): this.type = set(loss, value) + + /** + * Sets the value of param [[epsilon]]. + * Default is 1.35. + * + * @group setExpertParam + */ + @Since("2.3.0") + def setEpsilon(value: Double): this.type = set(epsilon, value) + + /** + * Sets the value of param [[maxBlockSizeInMB]]. + * Default is 0.0, then 1.0 MB will be chosen. + * + * @group expertSetParam + */ + @Since("3.1.0") + def setMaxBlockSizeInMB(value: Double): this.type = set(maxBlockSizeInMB, value) + + override protected def train( + dataset: Dataset[_]): LinearRegressionModel = { + val shim = LinearRegressionShim.create(uid) + shim.initShim(extractParamMap()) + shim.train(dataset) + } + + @Since("1.4.0") + override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra) +} + +@Since("1.6.0") +object LinearRegression extends DefaultParamsReadable[LinearRegression] { + + @Since("1.6.0") + override def load(path: String): LinearRegression = super.load(path) + + /** + * When using `LinearRegression.solver` == "normal", the solver must limit the number of + * features to at most this number. The entire covariance matrix X^T^X will be collected + * to the driver. This limit helps prevent memory overflow errors. + */ + @Since("2.1.0") + val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES + + /** String name for "auto". */ + private[regression] val Auto = "auto" + + /** String name for "normal". */ + private[regression] val Normal = "normal" + + /** String name for "l-bfgs". */ + private[regression] val LBFGS = "l-bfgs" + + /** Set of solvers that LinearRegression supports. */ + private[regression] val supportedSolvers = Array(Auto, Normal, LBFGS) + + /** String name for "squaredError". */ + private[regression] val SquaredError = "squaredError" + + /** String name for "huber". */ + private[regression] val Huber = "huber" + + /** Set of loss function names that LinearRegression supports. */ + private[regression] val supportedLosses = Array(SquaredError, Huber) +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/HuberAggregator.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/HuberAggregator.scala new file mode 100644 index 000000000..35582dbc9 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/HuberAggregator.scala @@ -0,0 +1,250 @@ +/* + * 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.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ + +/** + * HuberAggregator computes the gradient and loss for a huber loss function, + * as used in robust regression for samples in sparse or dense vector in an online fashion. + * + * The huber loss function based on: + * Art B. Owen (2006), + * A robust hybrid of lasso and ridge regression. + * + * Two HuberAggregator can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * The huber loss function is given by + * + *
+ * $$ + * \begin{align} + * \min_{w, \sigma}\frac{1}{2n}{\sum_{i=1}^n\left(\sigma + + * H_m\left(\frac{X_{i}w - y_{i}}{\sigma}\right)\sigma\right) + \frac{1}{2}\lambda {||w||_2}^2} + * \end{align} + * $$ + *
+ * + * where + * + *
+ * $$ + * \begin{align} + * H_m(z) = \begin{cases} + * z^2, & \text {if } |z| < \epsilon, \\ + * 2\epsilon|z| - \epsilon^2, & \text{otherwise} + * \end{cases} + * \end{align} + * $$ + *
+ * + * It is advised to set the parameter $\epsilon$ to 1.35 to achieve 95% statistical efficiency + * for normally distributed data. Please refer to chapter 2 of + * + * A robust hybrid of lasso and ridge regression for more detail. + * + * @param fitIntercept Whether to fit an intercept term. + * @param epsilon The shape parameter to control the amount of robustness. + * @param bcFeaturesStd The broadcast standard deviation values of the features. + * @param bcParameters including three parts: the regression coefficients corresponding + * to the features, the intercept (if fitIntercept is ture) + * and the scale parameter (sigma). + */ +private[ml] class HuberAggregator( + fitIntercept: Boolean, + epsilon: Double, + bcFeaturesStd: Broadcast[Array[Double]])(bcParameters: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, HuberAggregator] { + + protected override val dim: Int = bcParameters.value.size + private val numFeatures = if (fitIntercept) dim - 2 else dim - 1 + private val sigma = bcParameters.value(dim - 1) + private val intercept = if (fitIntercept) bcParameters.value(dim - 2) else 0.0 + + // make transient so we do not serialize between aggregation stages + @transient private lazy val coefficients = bcParameters.value.toArray.take(numFeatures) + + /** + * Add a new training instance to this HuberAggregator, and update the loss and gradient + * of the objective function. + * + * @param instance The instance of data point to be added. + * @return This HuberAggregator object. + */ + def add(instance: Instance): HuberAggregator = { + instance match { case Instance(label, weight, features) => + require(numFeatures == features.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $numFeatures but got ${features.size}.") + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") + + if (weight == 0.0) return this + val localFeaturesStd = bcFeaturesStd.value + val localCoefficients = coefficients + val localGradientSumArray = gradientSumArray + + val margin = { + var sum = 0.0 + features.foreachNonZero { (index, value) => + if (localFeaturesStd(index) != 0.0) { + sum += localCoefficients(index) * (value / localFeaturesStd(index)) + } + } + if (fitIntercept) sum += intercept + sum + } + val linearLoss = label - margin + + if (math.abs(linearLoss) <= sigma * epsilon) { + lossSum += 0.5 * weight * (sigma + math.pow(linearLoss, 2.0) / sigma) + val linearLossDivSigma = linearLoss / sigma + + features.foreachNonZero { (index, value) => + if (localFeaturesStd(index) != 0.0) { + localGradientSumArray(index) += + -1.0 * weight * linearLossDivSigma * (value / localFeaturesStd(index)) + } + } + if (fitIntercept) { + localGradientSumArray(dim - 2) += -1.0 * weight * linearLossDivSigma + } + localGradientSumArray(dim - 1) += 0.5 * weight * (1.0 - math.pow(linearLossDivSigma, 2.0)) + } else { + val sign = if (linearLoss >= 0) -1.0 else 1.0 + lossSum += 0.5 * weight * + (sigma + 2.0 * epsilon * math.abs(linearLoss) - sigma * epsilon * epsilon) + + features.foreachNonZero { (index, value) => + if (localFeaturesStd(index) != 0.0) { + localGradientSumArray(index) += + weight * sign * epsilon * (value / localFeaturesStd(index)) + } + } + if (fitIntercept) { + localGradientSumArray(dim - 2) += weight * sign * epsilon + } + localGradientSumArray(dim - 1) += 0.5 * weight * (1.0 - epsilon * epsilon) + } + + weightSum += weight + this + } + } +} + + +/** + * BlockHuberAggregator computes the gradient and loss for Huber loss function + * as used in linear regression for blocks in sparse or dense matrix in an online fashion. + * + * Two BlockHuberAggregators can be merged together to have a summary of loss and gradient + * of the corresponding joint dataset. + * + * NOTE: The feature values are expected to be standardized before computation. + * + * @param fitIntercept Whether to fit an intercept term. + */ +private[ml] class BlockHuberAggregator( + fitIntercept: Boolean, + epsilon: Double)(bcParameters: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, BlockHuberAggregator] { + + protected override val dim: Int = bcParameters.value.size + private val numFeatures = if (fitIntercept) dim - 2 else dim - 1 + private val intercept = if (fitIntercept) bcParameters.value(dim - 2) else 0.0 + // make transient so we do not serialize between aggregation stages + @transient private lazy val linear = Vectors.dense(bcParameters.value.toArray.take(numFeatures)) + + /** + * Add a new training instance block to this BlockHuberAggregator, and update the loss and + * gradient of the objective function. + * + * @param block The instance block of data point to be added. + * @return This BlockHuberAggregator object. + */ + def add(block: InstanceBlock): BlockHuberAggregator = { + require(block.matrix.isTransposed) + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + + val size = block.size + val sigma = bcParameters.value(dim - 1) + + // vec here represents margins or dotProducts + val vec = if (fitIntercept) { + Vectors.dense(Array.fill(size)(intercept)).toDense + } else { + Vectors.zeros(size).toDense + } + BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) + + // in-place convert margins to multipliers + // then, vec represents multipliers + var sigmaGradSum = 0.0 + var localLossSum = 0.0 + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + if (weight > 0) { + val label = block.getLabel(i) + val margin = vec(i) + val linearLoss = label - margin + + if (math.abs(linearLoss) <= sigma * epsilon) { + localLossSum += 0.5 * weight * (sigma + math.pow(linearLoss, 2.0) / sigma) + val linearLossDivSigma = linearLoss / sigma + val multiplier = -1.0 * weight * linearLossDivSigma + vec.values(i) = multiplier + sigmaGradSum += 0.5 * weight * (1.0 - math.pow(linearLossDivSigma, 2.0)) + } else { + localLossSum += 0.5 * weight * + (sigma + 2.0 * epsilon * math.abs(linearLoss) - sigma * epsilon * epsilon) + val sign = if (linearLoss >= 0) -1.0 else 1.0 + val multiplier = weight * sign * epsilon + vec.values(i) = multiplier + sigmaGradSum += 0.5 * weight * (1.0 - epsilon * epsilon) + } + } else { vec.values(i) = 0.0 } + i += 1 + } + lossSum += localLossSum + weightSum += block.weightIter.sum + + block.matrix match { + case dm: DenseMatrix => + BLAS.nativeBLAS.dgemv("N", dm.numCols, dm.numRows, 1.0, dm.values, dm.numCols, + vec.values, 1, 1.0, gradientSumArray, 1) + + case sm: SparseMatrix => + val linearGradSumVec = Vectors.zeros(numFeatures).toDense + BLAS.gemv(1.0, sm.transpose, vec, 0.0, linearGradSumVec) + BLAS.getBLAS(numFeatures).daxpy(numFeatures, 1.0, linearGradSumVec.values, 1, + gradientSumArray, 1) + } + + gradientSumArray(dim - 1) += sigmaGradSum + if (fitIntercept) gradientSumArray(dim - 2) += vec.values.sum + + this + } +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LeastSquaresAggregator.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LeastSquaresAggregator.scala new file mode 100644 index 000000000..d5e1ea980 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LeastSquaresAggregator.scala @@ -0,0 +1,314 @@ +/* + * 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.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ + +/** + * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, + * as used in linear regression for samples in sparse or dense vector in an online fashion. + * + * Two LeastSquaresAggregator can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * For improving the convergence rate during the optimization process, and also preventing against + * features with very large variances exerting an overly large influence during model training, + * package like R's GLMNET performs the scaling to unit variance and removing the mean to reduce + * the condition number, and then trains the model in scaled space but returns the coefficients in + * the original scale. See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf + * + * However, we don't want to apply the `StandardScaler` on the training dataset, and then cache + * the standardized dataset since it will create a lot of overhead. As a result, we perform the + * scaling implicitly when we compute the objective function. The following is the mathematical + * derivation. + * + * Note that we don't deal with intercept by adding bias here, because the intercept + * can be computed using closed form after the coefficients are converged. + * See this discussion for detail. + * http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet + * + * When training with intercept enabled, + * The objective function in the scaled space is given by + * + *
+ * $$ + * L = 1/2n ||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2, + * $$ + *
+ * + * where $\bar{x_i}$ is the mean of $x_i$, $\hat{x_i}$ is the standard deviation of $x_i$, + * $\bar{y}$ is the mean of label, and $\hat{y}$ is the standard deviation of label. + * + * If we fitting the intercept disabled (that is forced through 0.0), + * we can use the same equation except we set $\bar{y}$ and $\bar{x_i}$ to 0 instead + * of the respective means. + * + * This can be rewritten as + * + *
+ * $$ + * \begin{align} + * L &= 1/2n ||\sum_i (w_i/\hat{x_i})x_i - \sum_i (w_i/\hat{x_i})\bar{x_i} - y / \hat{y} + * + \bar{y} / \hat{y}||^2 \\ + * &= 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 + * \end{align} + * $$ + *
+ * + * where $w_i^\prime$ is the effective coefficients defined by $w_i/\hat{x_i}$, offset is + * + *
+ * $$ + * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. + * $$ + *
+ * + * and diff is + * + *
+ * $$ + * \sum_i w_i^\prime x_i - y / \hat{y} + offset + * $$ + *
+ * + * Note that the effective coefficients and offset don't depend on training dataset, + * so they can be precomputed. + * + * Now, the first derivative of the objective function in scaled space is + * + *
+ * $$ + * \frac{\partial L}{\partial w_i} = diff/N (x_i - \bar{x_i}) / \hat{x_i} + * $$ + *
+ * + * However, $(x_i - \bar{x_i})$ will densify the computation, so it's not + * an ideal formula when the training dataset is sparse format. + * + * This can be addressed by adding the dense $\bar{x_i} / \hat{x_i}$ terms + * in the end by keeping the sum of diff. The first derivative of total + * objective function from all the samples is + * + * + *
+ * $$ + * \begin{align} + * \frac{\partial L}{\partial w_i} &= + * 1/N \sum_j diff_j (x_{ij} - \bar{x_i}) / \hat{x_i} \\ + * &= 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i} / \hat{x_i}) \\ + * &= 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + correction_i) + * \end{align} + * $$ + *
+ * + * where $correction_i = - diffSum \bar{x_i} / \hat{x_i}$ + * + * A simple math can show that diffSum is actually zero, so we don't even + * need to add the correction terms in the end. From the definition of diff, + * + *
+ * $$ + * \begin{align} + * diffSum &= \sum_j (\sum_i w_i(x_{ij} - \bar{x_i}) + * / \hat{x_i} - (y_j - \bar{y}) / \hat{y}) \\ + * &= N * (\sum_i w_i(\bar{x_i} - \bar{x_i}) / \hat{x_i} - (\bar{y} - \bar{y}) / \hat{y}) \\ + * &= 0 + * \end{align} + * $$ + *
+ * + * As a result, the first derivative of the total objective function only depends on + * the training dataset, which can be easily computed in distributed fashion, and is + * sparse format friendly. + * + *
+ * $$ + * \frac{\partial L}{\partial w_i} = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + * $$ + *
+ * + * @note The constructor is curried, since the cost function will repeatedly create new versions + * of this class for different coefficient vectors. + * + * @param labelStd The standard deviation value of the label. + * @param labelMean The mean value of the label. + * @param fitIntercept Whether to fit an intercept term. + * @param bcFeaturesStd The broadcast standard deviation values of the features. + * @param bcFeaturesMean The broadcast mean values of the features. + * @param bcCoefficients The broadcast coefficients corresponding to the features. + */ +private[ml] class LeastSquaresAggregator( + labelStd: Double, + labelMean: Double, + fitIntercept: Boolean, + bcFeaturesStd: Broadcast[Array[Double]], + bcFeaturesMean: Broadcast[Array[Double]])(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, LeastSquaresAggregator] { + require(labelStd > 0.0, s"${this.getClass.getName} requires the label standard " + + s"deviation to be positive.") + + private val numFeatures = bcFeaturesStd.value.length + protected override val dim: Int = numFeatures + // make transient so we do not serialize between aggregation stages + @transient private lazy val featuresStd = bcFeaturesStd.value + @transient private lazy val effectiveCoefAndOffset = { + val coefficientsArray = bcCoefficients.value.toArray.clone() + val featuresMean = bcFeaturesMean.value + var sum = 0.0 + var i = 0 + val len = coefficientsArray.length + while (i < len) { + if (featuresStd(i) != 0.0) { + coefficientsArray(i) /= featuresStd(i) + sum += coefficientsArray(i) * featuresMean(i) + } else { + coefficientsArray(i) = 0.0 + } + i += 1 + } + val offset = if (fitIntercept) labelMean / labelStd - sum else 0.0 + (Vectors.dense(coefficientsArray), offset) + } + // do not use tuple assignment above because it will circumvent the @transient tag + @transient private lazy val effectiveCoefficientsVector = effectiveCoefAndOffset._1 + @transient private lazy val offset = effectiveCoefAndOffset._2 + + /** + * Add a new training instance to this LeastSquaresAggregator, and update the loss and gradient + * of the objective function. + * + * @param instance The instance of data point to be added. + * @return This LeastSquaresAggregator object. + */ + def add(instance: Instance): LeastSquaresAggregator = { + instance match { case Instance(label, weight, features) => + require(numFeatures == features.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $numFeatures but got ${features.size}.") + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") + + if (weight == 0.0) return this + + val diff = BLAS.dot(features, effectiveCoefficientsVector) - label / labelStd + offset + + if (diff != 0) { + val localGradientSumArray = gradientSumArray + val localFeaturesStd = featuresStd + features.foreachNonZero { (index, value) => + val fStd = localFeaturesStd(index) + if (fStd != 0.0) { + localGradientSumArray(index) += weight * diff * value / fStd + } + } + lossSum += weight * diff * diff / 2.0 + } + weightSum += weight + this + } + } +} + + +/** + * BlockLeastSquaresAggregator computes the gradient and loss for LeastSquares loss function + * as used in linear regression for blocks in sparse or dense matrix in an online fashion. + * + * Two BlockLeastSquaresAggregators can be merged together to have a summary of loss and gradient + * of the corresponding joint dataset. + * + * NOTE: The feature values are expected to be standardized before computation. + * + * @param bcCoefficients The coefficients corresponding to the features. + * @param fitIntercept Whether to fit an intercept term. + */ +private[ml] class BlockLeastSquaresAggregator( + labelStd: Double, + labelMean: Double, + fitIntercept: Boolean, + bcFeaturesStd: Broadcast[Array[Double]], + bcFeaturesMean: Broadcast[Array[Double]])(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, BlockLeastSquaresAggregator] { + require(labelStd > 0.0, s"${this.getClass.getName} requires the label standard " + + s"deviation to be positive.") + + private val numFeatures = bcFeaturesStd.value.length + protected override val dim: Int = numFeatures + // make transient so we do not serialize between aggregation stages + @transient private lazy val effectiveCoefAndOffset = { + val coefficientsArray = bcCoefficients.value.toArray.clone() + val featuresMean = bcFeaturesMean.value + val featuresStd = bcFeaturesStd.value + var sum = 0.0 + var i = 0 + val len = coefficientsArray.length + while (i < len) { + if (featuresStd(i) != 0.0) { + sum += coefficientsArray(i) / featuresStd(i) * featuresMean(i) + } else { + coefficientsArray(i) = 0.0 + } + i += 1 + } + val offset = if (fitIntercept) labelMean / labelStd - sum else 0.0 + (Vectors.dense(coefficientsArray), offset) + } + + /** + * Add a new training instance block to this BlockLeastSquaresAggregator, and update the loss + * and gradient of the objective function. + * + * @param block The instance block of data point to be added. + * @return This BlockLeastSquaresAggregator object. + */ + def add(block: InstanceBlock): BlockLeastSquaresAggregator = { + require(block.matrix.isTransposed) + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + + val size = block.size + val (effectiveCoefficientsVec, offset) = effectiveCoefAndOffset + + // vec here represents diffs + val vec = new DenseVector(Array.tabulate(size)(i => offset - block.getLabel(i) / labelStd)) + BLAS.gemv(1.0, block.matrix, effectiveCoefficientsVec, 1.0, vec) + + // in-place convert diffs to multipliers + // then, vec represents multipliers + var localLossSum = 0.0 + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + val diff = vec(i) + localLossSum += weight * diff * diff / 2 + val multiplier = weight * diff + vec.values(i) = multiplier + i += 1 + } + lossSum += localLossSum + weightSum += block.weightIter.sum + + val gradSumVec = new DenseVector(gradientSumArray) + BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) + + this + } +} diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LinearRegression.scala similarity index 58% rename from mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LinearRegression.scala index 2e342ea3d..aed92a156 100644 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark312/LinearRegression.scala @@ -15,12 +15,14 @@ * limitations under the License. */ -package org.apache.spark.ml.regression +package org.apache.spark.ml.regression.spark312 import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, FirstOrderMinimizer, LBFGS => BreezeLBFGS, LBFGSB => BreezeLBFGSB, OWLQN => BreezeOWLQN} import breeze.stats.distributions.StudentsT +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.regression.{LinearRegressionDALImpl, LinearRegressionShim} import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since @@ -47,6 +49,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.VersionUtils.majorMinorVersion +import org.apache.spark.ml.regression.{LinearRegression => SparkLinearRegression, _} /** * Params for linear regression. @@ -180,149 +183,148 @@ private[regression] trait LinearRegressionParams extends PredictorParams * Note: Fitting with huber loss only supports none and L2 regularization. */ @Since("1.3.0") -class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) - extends Regressor[Vector, LinearRegression, LinearRegressionModel] - with LinearRegressionParams with DefaultParamsWritable with Logging { +class LinearRegression @Since("1.3") (@Since("1.3.0") override val uid: String) + extends SparkLinearRegression with LinearRegressionShim { import LinearRegression._ @Since("1.4.0") def this() = this(Identifiable.randomUID("linReg")) - /** - * Set the regularization parameter. - * Default is 0.0. - * - * @group setParam - */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - - /** - * Set if we should fit the intercept. - * Default is true. - * - * @group setParam - */ - @Since("1.5.0") - def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - - /** - * Whether to standardize the training features before fitting the model. - * The coefficients of models will be always returned on the original scale, - * so it will be transparent for users. - * Default is true. - * - * @note With/without standardization, the models should be always converged - * to the same solution when no regularization is applied. In R's GLMNET package, - * the default behavior is true as well. - * - * @group setParam - */ - @Since("1.5.0") - def setStandardization(value: Boolean): this.type = set(standardization, value) - - /** - * Set the ElasticNet mixing parameter. - * For alpha = 0, the penalty is an L2 penalty. - * For alpha = 1, it is an L1 penalty. - * For alpha in (0,1), the penalty is a combination of L1 and L2. - * Default is 0.0 which is an L2 penalty. - * - * Note: Fitting with huber loss only supports None and L2 regularization, - * so throws exception if this param is non-zero value. - * - * @group setParam - */ - @Since("1.4.0") - def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - - /** - * Set the maximum number of iterations. - * Default is 100. - * - * @group setParam - */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** - * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. - * Default is 1E-6. - * - * @group setParam - */ - @Since("1.4.0") - def setTol(value: Double): this.type = set(tol, value) - - /** - * Whether to over-/under-sample training instances according to the given weights in weightCol. - * If not set or empty, all instances are treated equally (weight 1.0). - * Default is not set, so all instances have weight one. - * - * @group setParam - */ - @Since("1.6.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - /** - * Set the solver algorithm used for optimization. - * In case of linear regression, this can be "l-bfgs", "normal" and "auto". - * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton - * optimization method. - * - "normal" denotes using Normal Equation as an analytical solution to the linear regression - * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. - * - "auto" (default) means that the solver algorithm is selected automatically. - * The Normal Equations solver will be used when possible, but this will automatically fall - * back to iterative optimization methods when needed. - * - * Note: Fitting with huber loss doesn't support normal solver, - * so throws exception if this param was set with "normal". - * @group setParam - */ - @Since("1.6.0") - def setSolver(value: String): this.type = set(solver, value) - - /** - * Suggested depth for treeAggregate (greater than or equal to 2). - * If the dimensions of features or the number of partitions are large, - * this param could be adjusted to a larger size. - * Default is 2. - * - * @group expertSetParam - */ - @Since("2.1.0") - def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - - /** - * Sets the value of param [[loss]]. - * Default is "squaredError". - * - * @group setParam - */ - @Since("2.3.0") - def setLoss(value: String): this.type = set(loss, value) - - /** - * Sets the value of param [[epsilon]]. - * Default is 1.35. - * - * @group setExpertParam - */ - @Since("2.3.0") - def setEpsilon(value: Double): this.type = set(epsilon, value) - - /** - * Sets the value of param [[maxBlockSizeInMB]]. - * Default is 0.0, then 1.0 MB will be chosen. - * - * @group expertSetParam - */ - @Since("3.1.0") - def setMaxBlockSizeInMB(value: Double): this.type = set(maxBlockSizeInMB, value) - - override protected def train( +// /** +// * Set the regularization parameter. +// * Default is 0.0. +// * +// * @group setParam +// */ +// @Since("1.3.0") +// def setRegParam(value: Double): this.type = set(regParam, value) +// +// /** +// * Set if we should fit the intercept. +// * Default is true. +// * +// * @group setParam +// */ +// @Since("1.5.0") +// def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) +// +// /** +// * Whether to standardize the training features before fitting the model. +// * The coefficients of models will be always returned on the original scale, +// * so it will be transparent for users. +// * Default is true. +// * +// * @note With/without standardization, the models should be always converged +// * to the same solution when no regularization is applied. In R's GLMNET package, +// * the default behavior is true as well. +// * +// * @group setParam +// */ +// @Since("1.5.0") +// def setStandardization(value: Boolean): this.type = set(standardization, value) +// +// /** +// * Set the ElasticNet mixing parameter. +// * For alpha = 0, the penalty is an L2 penalty. +// * For alpha = 1, it is an L1 penalty. +// * For alpha in (0,1), the penalty is a combination of L1 and L2. +// * Default is 0.0 which is an L2 penalty. +// * +// * Note: Fitting with huber loss only supports None and L2 regularization, +// * so throws exception if this param is non-zero value. +// * +// * @group setParam +// */ +// @Since("1.4.0") +// def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) +// +// /** +// * Set the maximum number of iterations. +// * Default is 100. +// * +// * @group setParam +// */ +// @Since("1.3.0") +// def setMaxIter(value: Int): this.type = set(maxIter, value) +// +// /** +// * Set the convergence tolerance of iterations. +// * Smaller value will lead to higher accuracy with the cost of more iterations. +// * Default is 1E-6. +// * +// * @group setParam +// */ +// @Since("1.4.0") +// def setTol(value: Double): this.type = set(tol, value) +// +// /** +// * Whether to over-/under-sample training instances according to the given weights in weightCol. +// * If not set or empty, all instances are treated equally (weight 1.0). +// * Default is not set, so all instances have weight one. +// * +// * @group setParam +// */ +// @Since("1.6.0") +// def setWeightCol(value: String): this.type = set(weightCol, value) +// +// /** +// * Set the solver algorithm used for optimization. +// * In case of linear regression, this can be "l-bfgs", "normal" and "auto". +// * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton +// * optimization method. +// * - "normal" denotes using Normal Equation as an analytical solution to the linear regression +// * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. +// * - "auto" (default) means that the solver algorithm is selected automatically. +// * The Normal Equations solver will be used when possible, but this will automatically fall +// * back to iterative optimization methods when needed. +// * +// * Note: Fitting with huber loss doesn't support normal solver, +// * so throws exception if this param was set with "normal". +// * @group setParam +// */ +// @Since("1.6.0") +// def setSolver(value: String): this.type = set(solver, value) +// +// /** +// * Suggested depth for treeAggregate (greater than or equal to 2). +// * If the dimensions of features or the number of partitions are large, +// * this param could be adjusted to a larger size. +// * Default is 2. +// * +// * @group expertSetParam +// */ +// @Since("2.1.0") +// def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) +// +// /** +// * Sets the value of param [[loss]]. +// * Default is "squaredError". +// * +// * @group setParam +// */ +// @Since("2.3.0") +// def setLoss(value: String): this.type = set(loss, value) +// +// /** +// * Sets the value of param [[epsilon]]. +// * Default is 1.35. +// * +// * @group setExpertParam +// */ +// @Since("2.3.0") +// def setEpsilon(value: Double): this.type = set(epsilon, value) +// +// /** +// * Sets the value of param [[maxBlockSizeInMB]]. +// * Default is 0.0, then 1.0 MB will be chosen. +// * +// * @group expertSetParam +// */ +// @Since("3.1.0") +// def setMaxBlockSizeInMB(value: Double): this.type = set(maxBlockSizeInMB, value) + + override def train( dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) @@ -436,6 +438,10 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model.setSummary(Some(trainingSummary)) } + override def initShim(params: ParamMap): Unit = { + params.toSeq.foreach { paramMap.put(_) } + } + private def trainWithNormal( dataset: Dataset[_], instr: Instrumentation): LinearRegressionModel = { @@ -453,7 +459,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String executor_num, executor_cores) // Return same model as WeightedLeastSquaresModel - val model = optimizer.train(dataset, Some(instr)) + val model = optimizer.train(dataset) val lrModel = copyValues( new LinearRegressionModel(uid, model.coefficients, model.intercept)) @@ -710,85 +716,85 @@ object LinearRegression extends DefaultParamsReadable[LinearRegression] { private[regression] val supportedLosses = Array(SquaredError, Huber) } -/** - * Model produced by [[LinearRegression]]. - */ -@Since("1.3.0") -class LinearRegressionModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("2.0.0") val coefficients: Vector, - @Since("1.3.0") val intercept: Double, - @Since("2.3.0") val scale: Double) - extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams with GeneralMLWritable - with HasTrainingSummary[LinearRegressionTrainingSummary] { - - private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = - this(uid, coefficients, intercept, 1.0) - - override val numFeatures: Int = coefficients.size - - /** - * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("1.5.0") - override def summary: LinearRegressionTrainingSummary = super.summary - - /** - * Evaluates the model on a test dataset. - * - * @param dataset Test dataset to evaluate model on. - */ - @Since("2.0.0") - def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() - new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, - $(labelCol), $(featuresCol), summaryModel, Array(0.0)) - } - - /** - * If the prediction column is set returns the current model and prediction column, - * otherwise generates a new column and sets it as the prediction column on a new copy - * of the current model. - */ - private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { - $(predictionCol) match { - case "" => - val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString - (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) - case p => (this, p) - } - } - - - override def predict(features: Vector): Double = { - dot(features, coefficients) + intercept - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegressionModel = { - val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) - newModel.setSummary(trainingSummary).setParent(parent) - } - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - * This also does not save the [[parent]] currently. - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" - } -} +///** +// * Model produced by [[LinearRegression]]. +// */ +//@Since("1.3.0") +//class LinearRegressionModel private[ml] ( +// @Since("1.4.0") override val uid: String, +// @Since("2.0.0") val coefficients: Vector, +// @Since("1.3.0") val intercept: Double, +// @Since("2.3.0") val scale: Double) +// extends RegressionModel[Vector, LinearRegressionModel] +// with LinearRegressionParams with GeneralMLWritable +// with HasTrainingSummary[LinearRegressionTrainingSummary] { +// +// private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = +// this(uid, coefficients, intercept, 1.0) +// +// override val numFeatures: Int = coefficients.size +// +// /** +// * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is +// * thrown if `hasSummary` is false. +// */ +// @Since("1.5.0") +// override def summary: LinearRegressionTrainingSummary = super.summary +// +// /** +// * Evaluates the model on a test dataset. +// * +// * @param dataset Test dataset to evaluate model on. +// */ +// @Since("2.0.0") +// def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { +// // Handle possible missing or invalid prediction columns +// val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() +// new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, +// $(labelCol), $(featuresCol), summaryModel, Array(0.0)) +// } +// +// /** +// * If the prediction column is set returns the current model and prediction column, +// * otherwise generates a new column and sets it as the prediction column on a new copy +// * of the current model. +// */ +// private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { +// $(predictionCol) match { +// case "" => +// val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString +// (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) +// case p => (this, p) +// } +// } +// +// +// override def predict(features: Vector): Double = { +// dot(features, coefficients) + intercept +// } +// +// @Since("1.4.0") +// override def copy(extra: ParamMap): LinearRegressionModel = { +// val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) +// newModel.setSummary(trainingSummary).setParent(parent) +// } +// +// /** +// * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. +// * +// * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. +// * An option to save [[summary]] may be added in the future. +// * +// * This also does not save the [[parent]] currently. +// */ +// @Since("1.6.0") +// override def write: GeneralMLWriter = new GeneralMLWriter(this) +// +// @Since("3.0.0") +// override def toString: String = { +// s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" +// } +//} /** A writer for LinearRegression that handles the "internal" (or default) format */ private class InternalLinearRegressionModelWriter @@ -833,273 +839,273 @@ private class PMMLLinearRegressionModelWriter } } -@Since("1.6.0") -object LinearRegressionModel extends MLReadable[LinearRegressionModel] { - - @Since("1.6.0") - override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader - - @Since("1.6.0") - override def load(path: String): LinearRegressionModel = super.load(path) - - private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[LinearRegressionModel].getName - - override def load(path: String): LinearRegressionModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.format("parquet").load(dataPath) - val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) - val model = if (majorVersion < 2 || (majorVersion == 2 && minorVersion <= 2)) { - // Spark 2.2 and before - val Row(intercept: Double, coefficients: Vector) = - MLUtils.convertVectorColumnsToML(data, "coefficients") - .select("intercept", "coefficients") - .head() - new LinearRegressionModel(metadata.uid, coefficients, intercept) - } else { - // Spark 2.3 and later - val Row(intercept: Double, coefficients: Vector, scale: Double) = - data.select("intercept", "coefficients", "scale").head() - new LinearRegressionModel(metadata.uid, coefficients, intercept, scale) - } - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Linear regression training results. Currently, the training summary ignores the - * training weights except for the objective trace. - * - * @param predictions predictions output by the model's `transform` method. - * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. - */ -@Since("1.5.0") -class LinearRegressionTrainingSummary private[regression] ( - predictions: DataFrame, - predictionCol: String, - labelCol: String, - featuresCol: String, - model: LinearRegressionModel, - diagInvAtWA: Array[Double], - val objectiveHistory: Array[Double]) - extends LinearRegressionSummary( - predictions, - predictionCol, - labelCol, - featuresCol, - model, - diagInvAtWA) { - - /** - * Number of training iterations until termination - * - * This value is only available when using the "l-bfgs" solver. - * - * @see `LinearRegression.solver` - */ - @Since("1.5.0") - val totalIterations = { - assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") - objectiveHistory.length - 1 - } -} - -/** - * Linear regression results evaluated on a dataset. - * - * @param predictions predictions output by the model's `transform` method. - * @param predictionCol Field in "predictions" which gives the predicted value of the label at - * each instance. - * @param labelCol Field in "predictions" which gives the true label of each instance. - * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. - */ -@Since("1.5.0") -class LinearRegressionSummary private[regression] ( - @transient val predictions: DataFrame, - val predictionCol: String, - val labelCol: String, - val featuresCol: String, - private val privateModel: LinearRegressionModel, - private val diagInvAtWA: Array[Double]) extends Serializable { - - @transient private val metrics = { - val weightCol = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - lit(1.0) - } else { - col(privateModel.getWeightCol).cast(DoubleType) - } - - new RegressionMetrics( - predictions - .select(col(predictionCol), col(labelCol).cast(DoubleType), weightCol) - .rdd - .map { case Row(pred: Double, label: Double, weight: Double) => (pred, label, weight) }, - !privateModel.getFitIntercept) - } - - /** - * Returns the explained variance regression score. - * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) - * Reference: - * Wikipedia explain variation - */ - @Since("1.5.0") - val explainedVariance: Double = metrics.explainedVariance - - /** - * Returns the mean absolute error, which is a risk function corresponding to the - * expected value of the absolute error loss or l1-norm loss. - */ - @Since("1.5.0") - val meanAbsoluteError: Double = metrics.meanAbsoluteError - - /** - * Returns the mean squared error, which is a risk function corresponding to the - * expected value of the squared error loss or quadratic loss. - */ - @Since("1.5.0") - val meanSquaredError: Double = metrics.meanSquaredError - - /** - * Returns the root mean squared error, which is defined as the square root of - * the mean squared error. - */ - @Since("1.5.0") - val rootMeanSquaredError: Double = metrics.rootMeanSquaredError - - /** - * Returns R^2^, the coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - */ - @Since("1.5.0") - val r2: Double = metrics.r2 - - /** - * Returns Adjusted R^2^, the adjusted coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - */ - @Since("2.3.0") - val r2adj: Double = { - val interceptDOF = if (privateModel.getFitIntercept) 1 else 0 - 1 - (1 - r2) * (numInstances - interceptDOF) / - (numInstances - privateModel.coefficients.size - interceptDOF) - } - - /** Residuals (label - predicted value) */ - @Since("1.5.0") - @transient lazy val residuals: DataFrame = { - val t = udf { (pred: Double, label: Double) => label - pred } - predictions.select(t(col(predictionCol), col(labelCol)).as("residuals")) - } - - /** Number of instances in DataFrame predictions */ - lazy val numInstances: Long = metrics.count - - /** Degrees of freedom */ - @Since("2.2.0") - val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { - numInstances - privateModel.coefficients.size - 1 - } else { - numInstances - privateModel.coefficients.size - } - - /** - * The weighted residuals, the usual residuals rescaled by - * the square root of the instance weights. - */ - lazy val devianceResiduals: Array[Double] = { - val weighted = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - lit(1.0) - } else { - sqrt(col(privateModel.getWeightCol)) - } - val dr = predictions - .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) - .multiply(weighted).as("weightedResiduals")) - .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) - .first() - Array(dr.getDouble(0), dr.getDouble(1)) - } - - /** - * Standard error of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val coefficientStandardErrors: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No Std. Error of coefficients available for this LinearRegressionModel") - } else { - val rss = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - meanSquaredError * numInstances - } else { - val t = udf { (pred: Double, label: Double, weight: Double) => - math.pow(label - pred, 2.0) * weight } - predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), - col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) - } - val sigma2 = rss / degreesOfFreedom - diagInvAtWA.map(_ * sigma2).map(math.sqrt) - } - } - - /** - * T-statistic of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val tValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No t-statistic available for this LinearRegressionModel") - } else { - val estimate = if (privateModel.getFitIntercept) { - Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) - } else { - privateModel.coefficients.toArray - } - estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } - } - } - - /** - * Two-sided p-value of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val pValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No p-value available for this LinearRegressionModel") - } else { - tValues.map { x => 2.0 * (1.0 - StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) } - } - } - -} +//@Since("1.6.0") +//object LinearRegressionModel extends MLReadable[LinearRegressionModel] { +// +// @Since("1.6.0") +// override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader +// +// @Since("1.6.0") +// override def load(path: String): LinearRegressionModel = super.load(path) +// +// private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { +// +// /** Checked against metadata when loading model */ +// private val className = classOf[LinearRegressionModel].getName +// +// override def load(path: String): LinearRegressionModel = { +// val metadata = DefaultParamsReader.loadMetadata(path, sc, className) +// +// val dataPath = new Path(path, "data").toString +// val data = sparkSession.read.format("parquet").load(dataPath) +// val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) +// val model = if (majorVersion < 2 || (majorVersion == 2 && minorVersion <= 2)) { +// // Spark 2.2 and before +// val Row(intercept: Double, coefficients: Vector) = +// MLUtils.convertVectorColumnsToML(data, "coefficients") +// .select("intercept", "coefficients") +// .head() +// new LinearRegressionModel(metadata.uid, coefficients, intercept) +// } else { +// // Spark 2.3 and later +// val Row(intercept: Double, coefficients: Vector, scale: Double) = +// data.select("intercept", "coefficients", "scale").head() +// new LinearRegressionModel(metadata.uid, coefficients, intercept, scale) +// } +// +// metadata.getAndSetParams(model) +// model +// } +// } +//} + +///** +// * Linear regression training results. Currently, the training summary ignores the +// * training weights except for the objective trace. +// * +// * @param predictions predictions output by the model's `transform` method. +// * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. +// */ +//@Since("1.5.0") +//class LinearRegressionTrainingSummary private[regression] ( +// predictions: DataFrame, +// predictionCol: String, +// labelCol: String, +// featuresCol: String, +// model: LinearRegressionModel, +// diagInvAtWA: Array[Double], +// val objectiveHistory: Array[Double]) +// extends LinearRegressionSummary( +// predictions, +// predictionCol, +// labelCol, +// featuresCol, +// model, +// diagInvAtWA) { +// +// /** +// * Number of training iterations until termination +// * +// * This value is only available when using the "l-bfgs" solver. +// * +// * @see `LinearRegression.solver` +// */ +// @Since("1.5.0") +// val totalIterations = { +// assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") +// objectiveHistory.length - 1 +// } +//} + +///** +// * Linear regression results evaluated on a dataset. +// * +// * @param predictions predictions output by the model's `transform` method. +// * @param predictionCol Field in "predictions" which gives the predicted value of the label at +// * each instance. +// * @param labelCol Field in "predictions" which gives the true label of each instance. +// * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. +// */ +//@Since("1.5.0") +//class LinearRegressionSummary private[regression] ( +// @transient val predictions: DataFrame, +// val predictionCol: String, +// val labelCol: String, +// val featuresCol: String, +// private val privateModel: LinearRegressionModel, +// private val diagInvAtWA: Array[Double]) extends Serializable { +// +// @transient private val metrics = { +// val weightCol = +// if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { +// lit(1.0) +// } else { +// col(privateModel.getWeightCol).cast(DoubleType) +// } +// +// new RegressionMetrics( +// predictions +// .select(col(predictionCol), col(labelCol).cast(DoubleType), weightCol) +// .rdd +// .map { case Row(pred: Double, label: Double, weight: Double) => (pred, label, weight) }, +// !privateModel.getFitIntercept) +// } +// +// /** +// * Returns the explained variance regression score. +// * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) +// * Reference: +// * Wikipedia explain variation +// */ +// @Since("1.5.0") +// val explainedVariance: Double = metrics.explainedVariance +// +// /** +// * Returns the mean absolute error, which is a risk function corresponding to the +// * expected value of the absolute error loss or l1-norm loss. +// */ +// @Since("1.5.0") +// val meanAbsoluteError: Double = metrics.meanAbsoluteError +// +// /** +// * Returns the mean squared error, which is a risk function corresponding to the +// * expected value of the squared error loss or quadratic loss. +// */ +// @Since("1.5.0") +// val meanSquaredError: Double = metrics.meanSquaredError +// +// /** +// * Returns the root mean squared error, which is defined as the square root of +// * the mean squared error. +// */ +// @Since("1.5.0") +// val rootMeanSquaredError: Double = metrics.rootMeanSquaredError +// +// /** +// * Returns R^2^, the coefficient of determination. +// * Reference: +// * Wikipedia coefficient of determination +// */ +// @Since("1.5.0") +// val r2: Double = metrics.r2 +// +// /** +// * Returns Adjusted R^2^, the adjusted coefficient of determination. +// * Reference: +// * Wikipedia coefficient of determination +// */ +// @Since("2.3.0") +// val r2adj: Double = { +// val interceptDOF = if (privateModel.getFitIntercept) 1 else 0 +// 1 - (1 - r2) * (numInstances - interceptDOF) / +// (numInstances - privateModel.coefficients.size - interceptDOF) +// } +// +// /** Residuals (label - predicted value) */ +// @Since("1.5.0") +// @transient lazy val residuals: DataFrame = { +// val t = udf { (pred: Double, label: Double) => label - pred } +// predictions.select(t(col(predictionCol), col(labelCol)).as("residuals")) +// } +// +// /** Number of instances in DataFrame predictions */ +// lazy val numInstances: Long = metrics.count +// +// /** Degrees of freedom */ +// @Since("2.2.0") +// val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { +// numInstances - privateModel.coefficients.size - 1 +// } else { +// numInstances - privateModel.coefficients.size +// } +// +// /** +// * The weighted residuals, the usual residuals rescaled by +// * the square root of the instance weights. +// */ +// lazy val devianceResiduals: Array[Double] = { +// val weighted = +// if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { +// lit(1.0) +// } else { +// sqrt(col(privateModel.getWeightCol)) +// } +// val dr = predictions +// .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) +// .multiply(weighted).as("weightedResiduals")) +// .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) +// .first() +// Array(dr.getDouble(0), dr.getDouble(1)) +// } +// +// /** +// * Standard error of estimated coefficients and intercept. +// * This value is only available when using the "normal" solver. +// * +// * If `LinearRegression.fitIntercept` is set to true, +// * then the last element returned corresponds to the intercept. +// * +// * @see `LinearRegression.solver` +// */ +// lazy val coefficientStandardErrors: Array[Double] = { +// if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { +// throw new UnsupportedOperationException( +// "No Std. Error of coefficients available for this LinearRegressionModel") +// } else { +// val rss = +// if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { +// meanSquaredError * numInstances +// } else { +// val t = udf { (pred: Double, label: Double, weight: Double) => +// math.pow(label - pred, 2.0) * weight } +// predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), +// col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) +// } +// val sigma2 = rss / degreesOfFreedom +// diagInvAtWA.map(_ * sigma2).map(math.sqrt) +// } +// } +// +// /** +// * T-statistic of estimated coefficients and intercept. +// * This value is only available when using the "normal" solver. +// * +// * If `LinearRegression.fitIntercept` is set to true, +// * then the last element returned corresponds to the intercept. +// * +// * @see `LinearRegression.solver` +// */ +// lazy val tValues: Array[Double] = { +// if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { +// throw new UnsupportedOperationException( +// "No t-statistic available for this LinearRegressionModel") +// } else { +// val estimate = if (privateModel.getFitIntercept) { +// Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) +// } else { +// privateModel.coefficients.toArray +// } +// estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } +// } +// } +// +// /** +// * Two-sided p-value of estimated coefficients and intercept. +// * This value is only available when using the "normal" solver. +// * +// * If `LinearRegression.fitIntercept` is set to true, +// * then the last element returned corresponds to the intercept. +// * +// * @see `LinearRegression.solver` +// */ +// lazy val pValues: Array[Double] = { +// if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { +// throw new UnsupportedOperationException( +// "No p-value available for this LinearRegressionModel") +// } else { +// tValues.map { x => 2.0 * (1.0 - StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) } +// } +// } +// +//} diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark320/LinearRegression.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark320/LinearRegression.scala new file mode 100644 index 000000000..203a332ca --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/regression/spark320/LinearRegression.scala @@ -0,0 +1,1116 @@ +/* + * 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.regression.spark320 + +import scala.collection.mutable +import breeze.linalg.{DenseVector => BDV} +import breeze.optimize.{CachedDiffFunction, DiffFunction, FirstOrderMinimizer, LBFGS => BreezeLBFGS, LBFGSB => BreezeLBFGSB, OWLQN => BreezeOWLQN} +import breeze.stats.distributions.StudentsT +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.regression.{LinearRegressionDALImpl, LinearRegressionShim} +import org.apache.hadoop.fs.Path +import org.apache.spark.SparkException +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{PipelineStage, PredictorParams} +import org.apache.spark.ml.feature._ +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.optim.WeightedLeastSquares +import org.apache.spark.ml.optim.aggregator._ +import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.stat._ +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.mllib.evaluation.RegressionMetrics +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.regression.{LinearRegressionModel => OldLinearRegressionModel} +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.VersionUtils.majorMinorVersion +import org.apache.spark.ml.regression.{LinearRegression => SparkLinearRegression, _} + +/** + * Params for linear regression. + */ +private[regression] trait LinearRegressionParams extends PredictorParams + with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol + with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver + with HasAggregationDepth with HasLoss with HasMaxBlockSizeInMB { + + import LinearRegression._ + + /** + * The solver algorithm for optimization. + * Supported options: "l-bfgs", "normal" and "auto". + * Default: "auto" + * + * @group param + */ + @Since("1.6.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default auto)", + ParamValidators.inArray[String](supportedSolvers)) + + /** + * The loss function to be optimized. + * Supported options: "squaredError" and "huber". + * Default: "squaredError" + * + * @group param + */ + @Since("2.3.0") + final override val loss: Param[String] = new Param[String](this, "loss", "The loss function to" + + s" be optimized. Supported options: ${supportedLosses.mkString(", ")}. (Default squaredError)", + ParamValidators.inArray[String](supportedLosses)) + + /** + * The shape parameter to control the amount of robustness. Must be > 1.0. + * At larger values of epsilon, the huber criterion becomes more similar to least squares + * regression; for small values of epsilon, the criterion is more similar to L1 regression. + * Default is 1.35 to get as much robustness as possible while retaining + * 95% statistical efficiency for normally distributed data. It matches sklearn + * HuberRegressor and is "M" from + * A robust hybrid of lasso and ridge regression. + * Only valid when "loss" is "huber". + * + * @group expertParam + */ + @Since("2.3.0") + final val epsilon = new DoubleParam(this, "epsilon", "The shape parameter to control the " + + "amount of robustness. Must be > 1.0.", ParamValidators.gt(1.0)) + + /** @group getExpertParam */ + @Since("2.3.0") + def getEpsilon: Double = $(epsilon) + + setDefault(regParam -> 0.0, fitIntercept -> true, standardization -> true, + elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, solver -> Auto, + aggregationDepth -> 2, loss -> SquaredError, epsilon -> 1.35, maxBlockSizeInMB -> 0.0) + + override protected def validateAndTransformSchema( + schema: StructType, + fitting: Boolean, + featuresDataType: DataType): StructType = { + if (fitting) { + if ($(loss) == Huber) { + require($(solver)!= Normal, "LinearRegression with huber loss doesn't support " + + "normal solver, please change solver to auto or l-bfgs.") + require($(elasticNetParam) == 0.0, "LinearRegression with huber loss only supports " + + s"L2 regularization, but got elasticNetParam = $getElasticNetParam.") + } + } + super.validateAndTransformSchema(schema, fitting, featuresDataType) + } +} + +/** + * Linear regression. + * + * The learning objective is to minimize the specified loss function, with regularization. + * This supports two kinds of loss: + * - squaredError (a.k.a squared loss) + * - huber (a hybrid of squared error for relatively small errors and absolute error for + * relatively large ones, and we estimate the scale parameter from training data) + * + * This supports multiple types of regularization: + * - none (a.k.a. ordinary least squares) + * - L2 (ridge regression) + * - L1 (Lasso) + * - L2 + L1 (elastic net) + * + * The squared error objective function is: + * + *
+ * $$ + * \begin{align} + * \min_{w}\frac{1}{2n}{\sum_{i=1}^n(X_{i}w - y_{i})^{2} + + * \lambda\left[\frac{1-\alpha}{2}{||w||_{2}}^{2} + \alpha{||w||_{1}}\right]} + * \end{align} + * $$ + *
+ * + * The huber objective function is: + * + *
+ * $$ + * \begin{align} + * \min_{w, \sigma}\frac{1}{2n}{\sum_{i=1}^n\left(\sigma + + * H_m\left(\frac{X_{i}w - y_{i}}{\sigma}\right)\sigma\right) + \frac{1}{2}\lambda {||w||_2}^2} + * \end{align} + * $$ + *
+ * + * where + * + *
+ * $$ + * \begin{align} + * H_m(z) = \begin{cases} + * z^2, & \text {if } |z| < \epsilon, \\ + * 2\epsilon|z| - \epsilon^2, & \text{otherwise} + * \end{cases} + * \end{align} + * $$ + *
+ * + * Since 3.1.0, it supports stacking instances into blocks and using GEMV for + * better performance. + * The block size will be 1.0 MB, if param maxBlockSizeInMB is set 0.0 by default. + * + * Note: Fitting with huber loss only supports none and L2 regularization. + */ +@Since("1.3.0") +class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) + extends SparkLinearRegression with LinearRegressionShim { + + import LinearRegression._ + + @Since("1.4.0") + def this() = this(Identifiable.randomUID("linReg")) + +// /** +// * Set the regularization parameter. +// * Default is 0.0. +// * +// * @group setParam +// */ +// @Since("1.3.0") +// def setRegParam(value: Double): this.type = set(regParam, value) +// +// /** +// * Set if we should fit the intercept. +// * Default is true. +// * +// * @group setParam +// */ +// @Since("1.5.0") +// def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) +// +// /** +// * Whether to standardize the training features before fitting the model. +// * The coefficients of models will be always returned on the original scale, +// * so it will be transparent for users. +// * Default is true. +// * +// * @note With/without standardization, the models should be always converged +// * to the same solution when no regularization is applied. In R's GLMNET package, +// * the default behavior is true as well. +// * +// * @group setParam +// */ +// @Since("1.5.0") +// def setStandardization(value: Boolean): this.type = set(standardization, value) +// +// /** +// * Set the ElasticNet mixing parameter. +// * For alpha = 0, the penalty is an L2 penalty. +// * For alpha = 1, it is an L1 penalty. +// * For alpha in (0,1), the penalty is a combination of L1 and L2. +// * Default is 0.0 which is an L2 penalty. +// * +// * Note: Fitting with huber loss only supports None and L2 regularization, +// * so throws exception if this param is non-zero value. +// * +// * @group setParam +// */ +// @Since("1.4.0") +// def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) +// +// /** +// * Set the maximum number of iterations. +// * Default is 100. +// * +// * @group setParam +// */ +// @Since("1.3.0") +// def setMaxIter(value: Int): this.type = set(maxIter, value) +// +// /** +// * Set the convergence tolerance of iterations. +// * Smaller value will lead to higher accuracy with the cost of more iterations. +// * Default is 1E-6. +// * +// * @group setParam +// */ +// @Since("1.4.0") +// def setTol(value: Double): this.type = set(tol, value) +// +// /** +// * Whether to over-/under-sample training instances according to the given weights in weightCol. +// * If not set or empty, all instances are treated equally (weight 1.0). +// * Default is not set, so all instances have weight one. +// * +// * @group setParam +// */ +// @Since("1.6.0") +// def setWeightCol(value: String): this.type = set(weightCol, value) +// +// /** +// * Set the solver algorithm used for optimization. +// * In case of linear regression, this can be "l-bfgs", "normal" and "auto". +// * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton +// * optimization method. +// * - "normal" denotes using Normal Equation as an analytical solution to the linear regression +// * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. +// * - "auto" (default) means that the solver algorithm is selected automatically. +// * The Normal Equations solver will be used when possible, but this will automatically fall +// * back to iterative optimization methods when needed. +// * +// * Note: Fitting with huber loss doesn't support normal solver, +// * so throws exception if this param was set with "normal". +// * @group setParam +// */ +// @Since("1.6.0") +// def setSolver(value: String): this.type = set(solver, value) +// +// /** +// * Suggested depth for treeAggregate (greater than or equal to 2). +// * If the dimensions of features or the number of partitions are large, +// * this param could be adjusted to a larger size. +// * Default is 2. +// * +// * @group expertSetParam +// */ +// @Since("2.1.0") +// def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) +// +// /** +// * Sets the value of param [[loss]]. +// * Default is "squaredError". +// * +// * @group setParam +// */ +// @Since("2.3.0") +// def setLoss(value: String): this.type = set(loss, value) +// +// /** +// * Sets the value of param [[epsilon]]. +// * Default is 1.35. +// * +// * @group setExpertParam +// */ +// @Since("2.3.0") +// def setEpsilon(value: Double): this.type = set(epsilon, value) +// +// /** +// * Sets the value of param [[maxBlockSizeInMB]]. +// * Default is 0.0, then 1.0 MB will be chosen. +// * +// * @group expertSetParam +// */ +// @Since("3.1.0") +// def setMaxBlockSizeInMB(value: Double): this.type = set(maxBlockSizeInMB, value) + + override def train( + dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => + instr.logPipelineStage(this) + instr.logDataset(dataset) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, solver, tol, + elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth, loss, + epsilon, maxBlockSizeInMB) + + if (dataset.storageLevel != StorageLevel.NONE) { + instr.logWarning(s"Input instances will be standardized, blockified to blocks, and " + + s"then cached during training. Be careful of double caching!") + } + + // Extract the number of features before deciding optimization solver. + val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + instr.logNumFeatures(numFeatures) + + if ($(loss) == SquaredError && (($(solver) == Auto && + numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == Normal)) { + return trainWithNormal(dataset, instr) + } + + val instances = extractInstances(dataset) + .setName("training instances") + + val (summarizer, labelSummarizer) = Summarizer + .getRegressionSummarizers(instances, $(aggregationDepth), Seq("mean", "std", "count")) + + val yMean = labelSummarizer.mean(0) + val rawYStd = labelSummarizer.std(0) + + instr.logNumExamples(labelSummarizer.count) + instr.logNamedValue(Instrumentation.loggerTags.meanOfLabels, yMean) + instr.logNamedValue(Instrumentation.loggerTags.varianceOfLabels, rawYStd) + instr.logSumOfWeights(summarizer.weightSum) + + var actualBlockSizeInMB = $(maxBlockSizeInMB) + if (actualBlockSizeInMB == 0) { + actualBlockSizeInMB = InstanceBlock.DefaultBlockSizeInMB + require(actualBlockSizeInMB > 0, "inferred actual BlockSizeInMB must > 0") + instr.logNamedValue("actualBlockSizeInMB", actualBlockSizeInMB.toString) + } + + if (rawYStd == 0.0) { + if ($(fitIntercept) || yMean == 0.0) { + return trainWithConstantLabel(dataset, instr, numFeatures, yMean) + } else { + require($(regParam) == 0.0, "The standard deviation of the label is zero. " + + "Model cannot be regularized.") + instr.logWarning(s"The standard deviation of the label is zero. " + + "Consider setting fitIntercept=true.") + } + } + + // if y is constant (rawYStd is zero), then y cannot be scaled. In this case + // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. + val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) + val featuresMean = summarizer.mean.toArray + val featuresStd = summarizer.std.toArray + + if (!$(fitIntercept) && + (0 until numFeatures).exists(i => featuresStd(i) == 0.0 && featuresMean(i) != 0.0)) { + instr.logWarning("Fitting LinearRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + + "columns. This behavior is the same as R glmnet but different from LIBSVM.") + } + + // Since we implicitly do the feature scaling when we compute the cost function + // to improve the convergence, the effective regParam will be changed. + val effectiveRegParam = $(loss) match { + case SquaredError => $(regParam) / yStd + case Huber => $(regParam) + } + val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam + val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam + + val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 + val regularization = if (effectiveL2RegParam != 0.0) { + val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures + Some(new L2Regularization(effectiveL2RegParam, shouldApply, + if ($(standardization)) None else Some(getFeaturesStd))) + } else None + + val optimizer = createOptimizer(effectiveRegParam, effectiveL1RegParam, + numFeatures, featuresStd) + + val initialSolution = $(loss) match { + case SquaredError => + Array.ofDim[Double](numFeatures) + case Huber => + val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 + Array.fill(dim)(1.0) + } + + val (parameters, objectiveHistory) = + trainImpl(instances, actualBlockSizeInMB, yMean, yStd, + featuresMean, featuresStd, initialSolution, regularization, optimizer) + + if (parameters == null) { + val msg = s"${optimizer.getClass.getName} failed." + instr.logError(msg) + throw new SparkException(msg) + } + + val model = createModel(parameters, yMean, yStd, featuresMean, featuresStd) + // Handle possible missing or invalid prediction columns + val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() + + val trainingSummary = new LinearRegressionTrainingSummary( + summaryModel.transform(dataset), predictionColName, $(labelCol), $(featuresCol), + model, Array(0.0), objectiveHistory) + model.setSummary(Some(trainingSummary)) + } + + override def initShim(params: ParamMap): Unit = { + params.toSeq.foreach { paramMap.put(_) } + } + + private def trainWithNormal( + dataset: Dataset[_], + instr: Instrumentation): LinearRegressionModel = { + // oneDAL only support simple linear regression and ridge regression + val paramSupported = ($(regParam) == 0) || ($(regParam) != 0 && $(elasticNetParam) == 0) + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + dataset.sparkSession.sparkContext) + if (paramSupported && Utils.isOAPEnabled && isPlatformSupported) { + val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + logInfo(s"LinearRegressionDAL fit using $executor_num Executors") + + val optimizer = new LinearRegressionDALImpl($(fitIntercept), $(regParam), + elasticNetParam = $(elasticNetParam), $(standardization), true, + executor_num, executor_cores) + + // Return same model as WeightedLeastSquaresModel + val model = optimizer.train(dataset) + + val lrModel = copyValues( + new LinearRegressionModel(uid, model.coefficients, model.intercept)) + + val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() + + val trainingSummary = new LinearRegressionTrainingSummary( + summaryModel.transform(dataset), + predictionColName, + $(labelCol), + $(featuresCol), + summaryModel, + model.diagInvAtWA.toArray, + model.objectiveHistory) + + return lrModel.setSummary(Some(trainingSummary)) + } else { + // For low dimensional data, WeightedLeastSquares is more efficient since the + // training algorithm only requires one pass through the data. (SPARK-10668) + + val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), + elasticNetParam = $(elasticNetParam), $(standardization), true, + solverType = WeightedLeastSquares.Auto, maxIter = $(maxIter), tol = $(tol)) + val instances = extractInstances(dataset) + .setName("training instances") + val model = optimizer.fit(instances, instr = OptionalInstrumentation.create(instr)) + // When it is trained by WeightedLeastSquares, training summary does not + // attach returned model. + val lrModel = copyValues(new LinearRegressionModel(uid, model.coefficients, model.intercept)) + val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() + val trainingSummary = new LinearRegressionTrainingSummary( + summaryModel.transform(dataset), predictionColName, $(labelCol), $(featuresCol), + summaryModel, model.diagInvAtWA.toArray, model.objectiveHistory) + + lrModel.setSummary(Some(trainingSummary)) + } + } + + private def trainWithConstantLabel( + dataset: Dataset[_], + instr: Instrumentation, + numFeatures: Int, + yMean: Double): LinearRegressionModel = { + // If the rawYStd==0 and fitIntercept==true, then the intercept is yMean with + // zero coefficient; as a result, training is not needed. + // Also, if rawYStd==0 and yMean==0, all the coefficients are zero regardless of + // the fitIntercept. + if (yMean == 0.0) { + instr.logWarning(s"Mean and standard deviation of the label are zero, so the " + + s"coefficients and the intercept will all be zero; as a result, training is not " + + s"needed.") + } else { + instr.logWarning(s"The standard deviation of the label is zero, so the coefficients " + + s"will be zeros and the intercept will be the mean of the label; as a result, " + + s"training is not needed.") + } + val coefficients = Vectors.sparse(numFeatures, Seq.empty) + val intercept = yMean + + val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) + // Handle possible missing or invalid prediction columns + val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() + + val trainingSummary = new LinearRegressionTrainingSummary( + summaryModel.transform(dataset), predictionColName, $(labelCol), $(featuresCol), + model, Array(0.0), Array(0.0)) + + model.setSummary(Some(trainingSummary)) + } + + private def createOptimizer( + effectiveRegParam: Double, + effectiveL1RegParam: Double, + numFeatures: Int, + featuresStd: Array[Double]) = { + $(loss) match { + case SquaredError => + if ($(elasticNetParam) == 0.0 || effectiveRegParam == 0.0) { + new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) + } else { + val standardizationParam = $(standardization) + def effectiveL1RegFun = (index: Int) => { + if (standardizationParam) { + effectiveL1RegParam + } else { + // If `standardization` is false, we still standardize the data + // to improve the rate of convergence; as a result, we have to + // perform this reverse standardization by penalizing each component + // differently to get effectively the same objective function when + // the training dataset is not standardized. + if (featuresStd(index) != 0.0) effectiveL1RegParam / featuresStd(index) else 0.0 + } + } + new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, effectiveL1RegFun, $(tol)) + } + case Huber => + val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 + val lowerBounds = BDV[Double](Array.fill(dim)(Double.MinValue)) + // Optimize huber loss in space "\sigma > 0" + lowerBounds(dim - 1) = Double.MinPositiveValue + val upperBounds = BDV[Double](Array.fill(dim)(Double.MaxValue)) + new BreezeLBFGSB(lowerBounds, upperBounds, $(maxIter), 10, $(tol)) + } + } + + private def trainImpl( + instances: RDD[Instance], + actualBlockSizeInMB: Double, + yMean: Double, + yStd: Double, + featuresMean: Array[Double], + featuresStd: Array[Double], + initialSolution: Array[Double], + regularization: Option[L2Regularization], + optimizer: FirstOrderMinimizer[BDV[Double], DiffFunction[BDV[Double]]]) = { + val numFeatures = featuresStd.length + val inverseStd = featuresStd.map(std => if (std != 0) 1.0 / std else 0.0) + val scaledMean = Array.tabulate(numFeatures)(i => inverseStd(i) * featuresMean(i)) + val bcInverseStd = instances.context.broadcast(inverseStd) + val bcScaledMean = instances.context.broadcast(scaledMean) + + val standardized = instances.mapPartitions { iter => + val func = StandardScalerModel.getTransformFunc(Array.empty, bcInverseStd.value, false, true) + iter.map { case Instance(label, weight, vec) => Instance(label, weight, func(vec)) } + } + + val maxMemUsage = (actualBlockSizeInMB * 1024L * 1024L).ceil.toLong + val blocks = InstanceBlock.blokifyWithMaxMemUsage(standardized, maxMemUsage) + .persist(StorageLevel.MEMORY_AND_DISK) + .setName(s"$uid: training blocks (blockSizeInMB=$actualBlockSizeInMB)") + + if ($(fitIntercept) && $(loss) == Huber) { + // orginal `initialSolution` is for problem: + // y = f(w1 * x1 / std_x1, w2 * x2 / std_x2, ..., intercept) + // we should adjust it to the initial solution for problem: + // y = f(w1 * (x1 - avg_x1) / std_x1, w2 * (x2 - avg_x2) / std_x2, ..., intercept) + // NOTE: this is NOOP before we finally support model initialization + val adapt = BLAS.javaBLAS.ddot(numFeatures, initialSolution, 1, scaledMean, 1) + initialSolution(numFeatures) += adapt + } + + val costFun = $(loss) match { + case SquaredError => + val getAggregatorFunc = new LeastSquaresBlockAggregator(bcInverseStd, bcScaledMean, + $(fitIntercept), yStd, yMean)(_) + new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) + case Huber => + val getAggregatorFunc = new HuberBlockAggregator(bcInverseStd, bcScaledMean, + $(fitIntercept), $(epsilon))(_) + new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) + } + + val states = optimizer.iterations(new CachedDiffFunction(costFun), + new BDV(initialSolution)) + + /* + Note that in Linear Regression, the objective history (loss + regularization) returned + from optimizer is computed in the scaled space given by the following formula. +
+ $$ + L &= 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 + + regTerms \\ + $$ +
+ */ + val arrayBuilder = mutable.ArrayBuilder.make[Double] + var state: optimizer.State = null + while (states.hasNext) { + state = states.next() + arrayBuilder += state.adjustedValue + } + + blocks.unpersist() + bcInverseStd.destroy() + bcScaledMean.destroy() + + val solution = if (state == null) null else state.x.toArray + if ($(fitIntercept) && $(loss) == Huber && solution != null) { + // the final solution is for problem: + // y = f(w1 * (x1 - avg_x1) / std_x1, w2 * (x2 - avg_x2) / std_x2, ..., intercept) + // we should adjust it back for original problem: + // y = f(w1 * x1 / std_x1, w2 * x2 / std_x2, ..., intercept) + val adapt = BLAS.javaBLAS.ddot(numFeatures, solution, 1, scaledMean, 1) + solution(numFeatures) -= adapt + } + (solution, arrayBuilder.result) + } + + private def createModel( + solution: Array[Double], + yMean: Double, + yStd: Double, + featuresMean: Array[Double], + featuresStd: Array[Double]): LinearRegressionModel = { + val numFeatures = featuresStd.length + /* + The coefficients are trained in the scaled space; we're converting them back to + the original space. + */ + val multiplier = if ($(loss) == Huber) 1.0 else yStd + val rawCoefficients = Array.tabulate(numFeatures) { i => + if (featuresStd(i) != 0) solution(i) * multiplier / featuresStd(i) else 0.0 + } + + val intercept = if ($(fitIntercept)) { + $(loss) match { + case SquaredError => + /* + The intercept of squared error in R's GLMNET is computed using closed form + after the coefficients are converged. See the following discussion for detail. + http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet + */ + yMean - BLAS.dot(Vectors.dense(rawCoefficients), Vectors.dense(featuresMean)) + case Huber => solution(numFeatures) + } + } else 0.0 + + val coefficients = Vectors.dense(rawCoefficients).compressed + val scale = if ($(loss) == Huber) solution.last else 1.0 + copyValues(new LinearRegressionModel(uid, coefficients, intercept, scale)) + } + + @Since("1.4.0") + override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra) +} + +@Since("1.6.0") +object LinearRegression extends DefaultParamsReadable[LinearRegression] { + + @Since("1.6.0") + override def load(path: String): LinearRegression = super.load(path) + + /** + * When using `LinearRegression.solver` == "normal", the solver must limit the number of + * features to at most this number. The entire covariance matrix X^T^X will be collected + * to the driver. This limit helps prevent memory overflow errors. + */ + @Since("2.1.0") + val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES + + /** String name for "auto". */ + private[regression] val Auto = "auto" + + /** String name for "normal". */ + private[regression] val Normal = "normal" + + /** String name for "l-bfgs". */ + private[regression] val LBFGS = "l-bfgs" + + /** Set of solvers that LinearRegression supports. */ + private[regression] val supportedSolvers = Array(Auto, Normal, LBFGS) + + /** String name for "squaredError". */ + private[regression] val SquaredError = "squaredError" + + /** String name for "huber". */ + private[regression] val Huber = "huber" + + /** Set of loss function names that LinearRegression supports. */ + private[regression] val supportedLosses = Array(SquaredError, Huber) +} + +///** +// * Model produced by [[LinearRegression]]. +// */ +//@Since("1.3.0") +//class LinearRegressionModel private[ml] ( +// @Since("1.4.0") override val uid: String, +// @Since("2.0.0") val coefficients: Vector, +// @Since("1.3.0") val intercept: Double, +// @Since("2.3.0") val scale: Double) +// extends RegressionModel[Vector, LinearRegressionModel] +// with LinearRegressionParams with GeneralMLWritable +// with HasTrainingSummary[LinearRegressionTrainingSummary] { +// +// private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = +// this(uid, coefficients, intercept, 1.0) +// +// override val numFeatures: Int = coefficients.size +// +// /** +// * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is +// * thrown if `hasSummary` is false. +// */ +// @Since("1.5.0") +// override def summary: LinearRegressionTrainingSummary = super.summary +// +// /** +// * Evaluates the model on a test dataset. +// * +// * @param dataset Test dataset to evaluate model on. +// */ +// @Since("2.0.0") +// def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { +// // Handle possible missing or invalid prediction columns +// val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() +// new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, +// $(labelCol), $(featuresCol), summaryModel, Array(0.0)) +// } +// +// /** +// * If the prediction column is set returns the current model and prediction column, +// * otherwise generates a new column and sets it as the prediction column on a new copy +// * of the current model. +// */ +// private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { +// $(predictionCol) match { +// case "" => +// val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString +// (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) +// case p => (this, p) +// } +// } +// +// +// override def predict(features: Vector): Double = { +// BLAS.dot(features, coefficients) + intercept +// } +// +// @Since("1.4.0") +// override def copy(extra: ParamMap): LinearRegressionModel = { +// val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) +// newModel.setSummary(trainingSummary).setParent(parent) +// } +// +// /** +// * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. +// * +// * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. +// * An option to save [[summary]] may be added in the future. +// * +// * This also does not save the [[parent]] currently. +// */ +// @Since("1.6.0") +// override def write: GeneralMLWriter = new GeneralMLWriter(this) +// +// @Since("3.0.0") +// override def toString: String = { +// s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" +// } +//} + +/** A writer for LinearRegression that handles the "internal" (or default) format */ +private class InternalLinearRegressionModelWriter + extends MLWriterFormat with MLFormatRegister { + + override def format(): String = "internal" + override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" + + private case class Data(intercept: Double, coefficients: Vector, scale: Double) + + override def write(path: String, sparkSession: SparkSession, + optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { + val instance = stage.asInstanceOf[LinearRegressionModel] + val sc = sparkSession.sparkContext + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: intercept, coefficients, scale + val data = Data(instance.intercept, instance.coefficients, instance.scale) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } +} + +/** A writer for LinearRegression that handles the "pmml" format */ +private class PMMLLinearRegressionModelWriter + extends MLWriterFormat with MLFormatRegister { + + override def format(): String = "pmml" + + override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" + + private case class Data(intercept: Double, coefficients: Vector) + + override def write(path: String, sparkSession: SparkSession, + optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { + val sc = sparkSession.sparkContext + // Construct the MLLib model which knows how to write to PMML. + val instance = stage.asInstanceOf[LinearRegressionModel] + val oldModel = new OldLinearRegressionModel(instance.coefficients, instance.intercept) + // Save PMML + oldModel.toPMML(sc, path) + } +} + +@Since("1.6.0") +object LinearRegressionModel extends MLReadable[LinearRegressionModel] { + + @Since("1.6.0") + override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader + + @Since("1.6.0") + override def load(path: String): LinearRegressionModel = super.load(path) + + private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[LinearRegressionModel].getName + + override def load(path: String): LinearRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) + val model = if (majorVersion < 2 || (majorVersion == 2 && minorVersion <= 2)) { + // Spark 2.2 and before + val Row(intercept: Double, coefficients: Vector) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("intercept", "coefficients") + .head() + new LinearRegressionModel(metadata.uid, coefficients, intercept) + } else { + // Spark 2.3 and later + val Row(intercept: Double, coefficients: Vector, scale: Double) = + data.select("intercept", "coefficients", "scale").head() + new LinearRegressionModel(metadata.uid, coefficients, intercept, scale) + } + + metadata.getAndSetParams(model) + model + } + } +} + +///** +// * Linear regression training results. Currently, the training summary ignores the +// * training weights except for the objective trace. +// * +// * @param predictions predictions output by the model's `transform` method. +// * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. +// */ +//@Since("1.5.0") +//class LinearRegressionTrainingSummary private[regression] ( +// predictions: DataFrame, +// predictionCol: String, +// labelCol: String, +// featuresCol: String, +// model: LinearRegressionModel, +// diagInvAtWA: Array[Double], +// val objectiveHistory: Array[Double]) +// extends LinearRegressionSummary( +// predictions, +// predictionCol, +// labelCol, +// featuresCol, +// model, +// diagInvAtWA) { +// +// /** +// * Number of training iterations until termination +// * +// * This value is only available when using the "l-bfgs" solver. +// * +// * @see `LinearRegression.solver` +// */ +// @Since("1.5.0") +// val totalIterations = { +// assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") +// objectiveHistory.length - 1 +// } +//} + +///** +// * Linear regression results evaluated on a dataset. +// * +// * @param predictions predictions output by the model's `transform` method. +// * @param predictionCol Field in "predictions" which gives the predicted value of the label at +// * each instance. +// * @param labelCol Field in "predictions" which gives the true label of each instance. +// * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. +// */ +//@Since("1.5.0") +//class LinearRegressionSummary private[regression] ( +// @transient val predictions: DataFrame, +// val predictionCol: String, +// val labelCol: String, +// val featuresCol: String, +// private val privateModel: LinearRegressionModel, +// private val diagInvAtWA: Array[Double]) extends Serializable { +// +// @transient private val metrics = { +// val weightCol = +// if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { +// lit(1.0) +// } else { +// col(privateModel.getWeightCol).cast(DoubleType) +// } +// +// new RegressionMetrics( +// predictions +// .select(col(predictionCol), col(labelCol).cast(DoubleType), weightCol) +// .rdd +// .map { case Row(pred: Double, label: Double, weight: Double) => (pred, label, weight) }, +// !privateModel.getFitIntercept) +// } +// +// /** +// * Returns the explained variance regression score. +// * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) +// * Reference: +// * Wikipedia explain variation +// */ +// @Since("1.5.0") +// val explainedVariance: Double = metrics.explainedVariance +// +// /** +// * Returns the mean absolute error, which is a risk function corresponding to the +// * expected value of the absolute error loss or l1-norm loss. +// */ +// @Since("1.5.0") +// val meanAbsoluteError: Double = metrics.meanAbsoluteError +// +// /** +// * Returns the mean squared error, which is a risk function corresponding to the +// * expected value of the squared error loss or quadratic loss. +// */ +// @Since("1.5.0") +// val meanSquaredError: Double = metrics.meanSquaredError +// +// /** +// * Returns the root mean squared error, which is defined as the square root of +// * the mean squared error. +// */ +// @Since("1.5.0") +// val rootMeanSquaredError: Double = metrics.rootMeanSquaredError +// +// /** +// * Returns R^2^, the coefficient of determination. +// * Reference: +// * Wikipedia coefficient of determination +// */ +// @Since("1.5.0") +// val r2: Double = metrics.r2 +// +// /** +// * Returns Adjusted R^2^, the adjusted coefficient of determination. +// * Reference: +// * Wikipedia coefficient of determination +// */ +// @Since("2.3.0") +// val r2adj: Double = { +// val interceptDOF = if (privateModel.getFitIntercept) 1 else 0 +// 1 - (1 - r2) * (numInstances - interceptDOF) / +// (numInstances - privateModel.coefficients.size - interceptDOF) +// } +// +// /** Residuals (label - predicted value) */ +// @Since("1.5.0") +// @transient lazy val residuals: DataFrame = { +// val t = udf { (pred: Double, label: Double) => label - pred } +// predictions.select(t(col(predictionCol), col(labelCol)).as("residuals")) +// } +// +// /** Number of instances in DataFrame predictions */ +// lazy val numInstances: Long = metrics.count +// +// /** Degrees of freedom */ +// @Since("2.2.0") +// val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { +// numInstances - privateModel.coefficients.size - 1 +// } else { +// numInstances - privateModel.coefficients.size +// } +// +// /** +// * The weighted residuals, the usual residuals rescaled by +// * the square root of the instance weights. +// */ +// lazy val devianceResiduals: Array[Double] = { +// val weighted = +// if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { +// lit(1.0) +// } else { +// sqrt(col(privateModel.getWeightCol)) +// } +// val dr = predictions +// .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) +// .multiply(weighted).as("weightedResiduals")) +// .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) +// .first() +// Array(dr.getDouble(0), dr.getDouble(1)) +// } +// +// /** +// * Standard error of estimated coefficients and intercept. +// * This value is only available when using the "normal" solver. +// * +// * If `LinearRegression.fitIntercept` is set to true, +// * then the last element returned corresponds to the intercept. +// * +// * @see `LinearRegression.solver` +// */ +// lazy val coefficientStandardErrors: Array[Double] = { +// if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { +// throw new UnsupportedOperationException( +// "No Std. Error of coefficients available for this LinearRegressionModel") +// } else { +// val rss = +// if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { +// meanSquaredError * numInstances +// } else { +// val t = udf { (pred: Double, label: Double, weight: Double) => +// math.pow(label - pred, 2.0) * weight } +// predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), +// col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) +// } +// val sigma2 = rss / degreesOfFreedom +// diagInvAtWA.map(_ * sigma2).map(math.sqrt) +// } +// } +// +// /** +// * T-statistic of estimated coefficients and intercept. +// * This value is only available when using the "normal" solver. +// * +// * If `LinearRegression.fitIntercept` is set to true, +// * then the last element returned corresponds to the intercept. +// * +// * @see `LinearRegression.solver` +// */ +// lazy val tValues: Array[Double] = { +// if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { +// throw new UnsupportedOperationException( +// "No t-statistic available for this LinearRegressionModel") +// } else { +// val estimate = if (privateModel.getFitIntercept) { +// Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) +// } else { +// privateModel.coefficients.toArray +// } +// estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } +// } +// } +// +// /** +// * Two-sided p-value of estimated coefficients and intercept. +// * This value is only available when using the "normal" solver. +// * +// * If `LinearRegression.fitIntercept` is set to true, +// * then the last element returned corresponds to the intercept. +// * +// * @see `LinearRegression.solver` +// */ +// lazy val pValues: Array[Double] = { +// if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { +// throw new UnsupportedOperationException( +// "No p-value available for this LinearRegressionModel") +// } else { +// tValues.map { x => 2.0 * (1.0 - StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) } +// } +// } +// +//} + diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/stat/Correlation.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/stat/Correlation.scala similarity index 69% rename from mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/stat/Correlation.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/stat/Correlation.scala index 5b411011a..7f5079863 100644 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/stat/Correlation.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/stat/Correlation.scala @@ -17,16 +17,16 @@ package org.apache.spark.ml.stat +import scala.collection.JavaConverters._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.linalg.{SQLDataTypes, Vector} -import org.apache.spark.ml.util.Utils import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.stat.{Statistics => OldStatistics} import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.storage.StorageLevel - -import scala.collection.JavaConverters._ +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.stat.{CorrelationDALImpl, CorrelationShim} /** * API for correlation functions in MLlib, compatible with DataFrames and Datasets. @@ -35,11 +35,9 @@ import scala.collection.JavaConverters._ * to spark.ml's Vector types. */ @Since("2.2.0") -@Experimental object Correlation { /** - * :: Experimental :: * Compute the correlation matrix for the input Dataset of Vectors using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * @@ -51,7 +49,7 @@ object Correlation { * Supported: `pearson` (default), `spearman` * @return A dataframe that contains the correlation matrix of the column of vectors. This * dataframe contains a single row and a single column of name - * '$METHODNAME($COLUMN)'. + * `$METHODNAME($COLUMN)`. * @throws IllegalArgumentException if the column is not a valid column in the dataset, or if * the content of this column is not of type Vector. * @@ -69,36 +67,8 @@ object Correlation { */ @Since("2.2.0") def corr(dataset: Dataset[_], column: String, method: String): DataFrame = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - if (Utils.isOAPEnabled() && isPlatformSupported && method == "pearson") { - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - if (handlePersistence) { - dataset.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = dataset.select(column).rdd.map { - case Row(v: Vector) => v - } - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val matrix = new CorrelationDALImpl(executor_num, executor_cores) - .computeCorrelationMatrix(rdd) - val name = s"$method($column)" - val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) - val dataframe = dataset.sparkSession.createDataFrame(Seq(Row(matrix)).asJava, schema) - if (handlePersistence) { - dataset.unpersist() - } - dataframe - } else { - val rdd = dataset.select(column).rdd.map { - case Row(v: Vector) => OldVectors.fromML(v) - } - val oldM = OldStatistics.corr(rdd, method) - val name = s"$method($column)" - val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) - dataset.sparkSession.createDataFrame(Seq(Row(oldM.asML)).asJava, schema) - } + val shim = CorrelationShim.create() + shim.corr(dataset, column, method) } /** diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/stat/Correlation.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/stat/spark320/Correlation.scala similarity index 95% rename from mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/stat/Correlation.scala rename to mllib-dal/src/main/scala/org/apache/spark/ml/stat/spark320/Correlation.scala index 81e75e64d..a551c1502 100644 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/stat/Correlation.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/stat/spark320/Correlation.scala @@ -15,19 +15,20 @@ * limitations under the License. */ -package org.apache.spark.ml.stat +package org.apache.spark.ml.stat.spark320 -import scala.collection.JavaConverters._ - -import org.apache.spark.annotation.{Experimental, Since} +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.stat.{CorrelationDALImpl, CorrelationShim} +import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.{SQLDataTypes, Vector} -import org.apache.spark.ml.util.Utils import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.stat.{Statistics => OldStatistics} import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.storage.StorageLevel +import scala.collection.JavaConverters._ + /** * API for correlation functions in MLlib, compatible with DataFrames and Datasets. * @@ -35,7 +36,7 @@ import org.apache.spark.storage.StorageLevel * to spark.ml's Vector types. */ @Since("2.2.0") -object Correlation { +class Correlation extends CorrelationShim { /** * Compute the correlation matrix for the input Dataset of Vectors using the specified method. diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala similarity index 100% rename from mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/clustering/KMeans.scala rename to mllib-dal/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala new file mode 100644 index 000000000..fb3a9ddfa --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala @@ -0,0 +1,118 @@ +/* + * 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.mllib.stat + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util.Utils.getOneCCLIPPort +import org.apache.spark.ml.util.{OneCCL, OneDAL} +import org.apache.spark.mllib.stat.SummarizerResult +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary => Summary} +import org.apache.spark.rdd.RDD + +class SummarizerDALImpl( + val executorNum: Int, + val executorCores: Int) + extends Serializable with Logging { + + def computeSummarizerMatrix(data: RDD[Vector]): Summary = { + + val kvsIPPort = getOneCCLIPPort(data) + + val sparkContext = data.sparkContext + val useGPU = sparkContext.conf.getBoolean("spark.oap.mllib.useGPU", false) + + val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) + + val results = coalescedTables.mapPartitionsWithIndex { (rank, table) => + + val gpuIndices = if (useGPU) { + val resources = TaskContext.get().resources() + resources("gpu").addresses.map(_.toInt) + } else { + null + } + + val tableArr = table.next() + OneCCL.init(executorNum, rank, kvsIPPort) + + val computeStartTime = System.nanoTime() + + val result = new SummarizerResult() + cSummarizerTrainDAL( + tableArr, + executorNum, + executorCores, + useGPU, + gpuIndices, + result + ) + + val computeEndTime = System.nanoTime() + + val durationCompute = (computeEndTime - computeStartTime).toDouble / 1E9 + + logInfo(s"SummarizerDAL compute took ${durationCompute} secs") + + val ret = if (OneCCL.isRoot()) { + + val convResultStartTime = System.nanoTime() + val meanArray= OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.meanNumericTable)) + val varianceArray = OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.varianceNumericTable)) + val maxrray= OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.maximumNumericTable)) + val minArray = OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.minimumNumericTable)) + + val convResultEndTime = System.nanoTime() + + val durationCovResult = (convResultEndTime - convResultStartTime).toDouble / 1E9 + + logInfo(s"SummarizerDAL result conversion took ${durationCovResult} secs") + + Iterator((meanArray(0), varianceArray(0), maxrray(0), minArray(0))) + } else { + Iterator.empty + } + + OneCCL.cleanup() + + ret + }.collect() + + // Make sure there is only one result from rank 0 + assert(results.length == 1) + + val meanVector = results(0)._1 + val varianceVector = results(0)._2 + val maxVector = results(0)._3 + val minVector = results(0)._4 + + val summary = new MultivariateStatisticalDALSummary(OldVectors.fromML(meanVector), OldVectors.fromML(varianceVector) + , OldVectors.fromML(maxVector), OldVectors.fromML(minVector)) + + summary + } + + + @native private def cSummarizerTrainDAL(data: Long, + executor_num: Int, + executor_cores: Int, + useGPU: Boolean, + gpuIndices: Array[Int], + result: SummarizerResult): Long +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala deleted file mode 100644 index 95892e228..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ /dev/null @@ -1,677 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.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.{DataFrame, Dataset, Row} -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. - */ -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 sc = dataset.sparkSession.sparkContext - val model = if (Utils.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) - } - model - 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 spark = dataset.sparkSession - import spark.implicits._ - - val sc = spark.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"NaiveBayesDAL fit using $executor_num Executors") - - // 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) - - // 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 - } - - instr.logNumClasses(numClasses) - - val labeledPointsDS = dataset - .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) - - val model = new NaiveBayesDALImpl(uid, numClasses, - 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 - model.setOldLabels(labels) - - 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/spark-3.0.0/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/clustering/KMeans.scala deleted file mode 100644 index ce0979e9d..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ /dev/null @@ -1,507 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.clustering - -import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.Since -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.ml.util._ -import org.apache.spark.ml.{Estimator, Model, PipelineStage} -import org.apache.spark.mllib.clustering.{DistanceMeasure, VectorWithNorm, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorVersion - -import scala.collection.mutable - - -/** - * Common params for KMeans and KMeansModel - */ -private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFeaturesCol - with HasSeed with HasPredictionCol with HasTol with HasDistanceMeasure with HasWeightCol { - - /** - * The number of clusters to create (k). Must be > 1. Note that it is possible for fewer than - * k clusters to be returned, for example, if there are fewer than k distinct points to cluster. - * Default: 2. - * @group param - */ - @Since("1.5.0") - final val k = new IntParam(this, "k", "The number of clusters to create. " + - "Must be > 1.", ParamValidators.gt(1)) - - /** @group getParam */ - @Since("1.5.0") - def getK: Int = $(k) - - /** - * Param for the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - * @group expertParam - */ - @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + - "Supported options: 'random' and 'k-means||'.", - (value: String) => MLlibKMeans.validateInitMode(value)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitMode: String = $(initMode) - - /** - * Param for the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Must be > 0. Default: 2. - * @group expertParam - */ - @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + - "initialization mode. Must be > 0.", ParamValidators.gt(0)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitSteps: Int = $(initSteps) - - /** - * Validates and transforms the input schema. - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.validateVectorCompatibleColumn(schema, getFeaturesCol) - SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) - } -} - -/** - * Model fitted by KMeans. - * - * @param parentModel a model trained by spark.mllib.clustering.KMeans. - */ -@Since("1.5.0") -class KMeansModel private[ml] ( - @Since("1.5.0") override val uid: String, - private[clustering] val parentModel: MLlibKMeansModel) - extends Model[KMeansModel] with KMeansParams with GeneralMLWritable - with HasTrainingSummary[KMeansSummary] { - - @Since("3.0.0") - lazy val numFeatures: Int = parentModel.clusterCenters.head.size - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeansModel = { - val copied = copyValues(new KMeansModel(uid, parentModel), extra) - copied.setSummary(trainingSummary).setParent(this.parent) - } - - /** @group setParam */ - @Since("2.0.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("2.0.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val predictUDF = udf((vector: Vector) => predict(vector)) - - dataset.withColumn($(predictionCol), - predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)), - outputSchema($(predictionCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(predictionCol).nonEmpty) { - outputSchema = SchemaUtils.updateNumValues(outputSchema, - $(predictionCol), parentModel.k) - } - outputSchema - } - - @Since("3.0.0") - def predict(features: Vector): Int = parentModel.predict(features) - - @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[KMeansModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"KMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + - s"numFeatures=$numFeatures" - } - - /** - * Gets summary of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("2.0.0") - override def summary: KMeansSummary = super.summary -} - -/** Helper class for storing model data */ -private case class ClusterData(clusterIdx: Int, clusterCenter: Vector) - - -/** A writer for KMeans that handles the "internal" (or default) format */ -private class InternalKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: cluster centers - val data: Array[ClusterData] = instance.clusterCenters.zipWithIndex.map { - case (center, idx) => - ClusterData(idx, center) - } - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(data).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for KMeans that handles the "pmml" format */ -private class PMMLKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - instance.parentModel.toPMML(sc, path) - } -} - - -@Since("1.6.0") -object KMeansModel extends MLReadable[KMeansModel] { - - @Since("1.6.0") - override def read: MLReader[KMeansModel] = new KMeansModelReader - - @Since("1.6.0") - override def load(path: String): KMeansModel = super.load(path) - - /** - * We store all cluster centers in a single row and use this class to store model data by - * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. - */ - private case class OldData(clusterCenters: Array[OldVector]) - - private class KMeansModelReader extends MLReader[KMeansModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[KMeansModel].getName - - override def load(path: String): KMeansModel = { - // Import implicits for Dataset Encoder - val sparkSession = super.sparkSession - import sparkSession.implicits._ - - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - - val clusterCenters = if (majorVersion(metadata.sparkVersion) >= 2) { - val data: Dataset[ClusterData] = sparkSession.read.parquet(dataPath).as[ClusterData] - data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) - } else { - // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. - sparkSession.read.parquet(dataPath).as[OldData].head().clusterCenters - } - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) - metadata.getAndSetParams(model) - model - } - } -} - -/** - * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. - * - * @see Bahmani et al., Scalable k-means++. - */ -@Since("1.5.0") -class KMeans @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { - - setDefault( - k -> 2, - maxIter -> 20, - initMode -> MLlibKMeans.K_MEANS_PARALLEL, - initSteps -> 2, - tol -> 1e-4, - distanceMeasure -> DistanceMeasure.EUCLIDEAN) - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeans = defaultCopy(extra) - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("kmeans")) - - /** @group setParam */ - @Since("1.5.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitMode(value: String): this.type = set(initMode, value) - - /** @group expertSetParam */ - @Since("2.4.0") - def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitSteps(value: Int): this.type = set(initSteps, value) - - /** @group setParam */ - @Since("1.5.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.5.0") - def setTol(value: Double): this.type = set(tol, value) - - /** @group setParam */ - @Since("1.5.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** - * 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("3.0.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): KMeansModel = instrumented { instr => - transformSchema(dataset.schema, logging = true) - - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - val handleWeight = isDefined(weightCol) && $(weightCol).nonEmpty - val w = if (handleWeight) { - col($(weightCol)).cast(DoubleType) - } else { - lit(1.0) - } - - val instances: RDD[(Vector, Double)] = dataset - .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w).rdd.map { - case Row(point: Vector, weight: Double) => (point, weight) - } - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, featuresCol, predictionCol, k, initMode, initSteps, distanceMeasure, - maxIter, seed, tol, weightCol) - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - val useKMeansDAL = Utils.isOAPEnabled() && isPlatformSupported && - $(distanceMeasure) == "euclidean" && !handleWeight - - val model = if (useKMeansDAL) { - trainWithDAL(instances, handlePersistence) - } else { - trainWithML(instances, handlePersistence) - } - - val summary = new KMeansSummary( - model.transform(dataset), - $(predictionCol), - $(featuresCol), - $(k), - model.parentModel.numIter, - model.parentModel.trainingCost) - - model.setSummary(Some(summary)) - instr.logNamedValue("clusterSizes", summary.clusterSizes) - - model - } - - private def trainWithDAL(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - - val sc = instances.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"KMeansDAL fit using $executor_num Executors") - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString($(distanceMeasure)) - - // Use MLlibKMeans to initialize centers - val mllibKMeans = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - - val dataWithNorm = instances.map { - case (point: Vector, weight: Double) => new VectorWithNorm(point) - } - - // Cache for init - dataWithNorm.persist(StorageLevel.MEMORY_AND_DISK) - - val centersWithNorm = if ($(initMode) == "random") { - mllibKMeans.initRandom(dataWithNorm) - } else { - mllibKMeans.initKMeansParallel(dataWithNorm, distanceMeasureInstance) - } - - dataWithNorm.unpersist() - - val centers = centersWithNorm.map(_.vector) - - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - - val strInitMode = $(initMode) - logInfo(f"Initialization with $strInitMode took $initTimeInSeconds%.3f seconds.") - - if (handlePersistence) { - instances.persist(StorageLevel.MEMORY_AND_DISK) - } - - val inputData = instances.map { - case (point: Vector, weight: Double) => point - } - - val kmeansDAL = new KMeansDALImpl(getK, getMaxIter, getTol, - DistanceMeasure.EUCLIDEAN, centers, executor_num, executor_cores) - - val parentModel = kmeansDAL.train(inputData, Option(instr)) - - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - - if (handlePersistence) { - instances.unpersist() - } - - model - } - - private def trainWithML(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - if (handlePersistence) { - instances.persist(StorageLevel.MEMORY_AND_DISK) - } - - val oldVectorInstances = instances.map { - case (point: Vector, weight: Double) => (OldVectors.fromML(point), weight) - } - val algo = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - val parentModel = algo.runWithWeight(oldVectorInstances, Option(instr)) - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - - if (handlePersistence) { - instances.unpersist() - } - - model - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } -} - -@Since("1.6.0") -object KMeans extends DefaultParamsReadable[KMeans] { - - @Since("1.6.0") - override def load(path: String): KMeans = super.load(path) -} - -/** - * Summary of KMeans. - * - * @param predictions `DataFrame` produced by `KMeansModel.transform()`. - * @param predictionCol Name for column of predicted clusters in `predictions`. - * @param featuresCol Name for column of features in `predictions`. - * @param k Number of clusters. - * @param numIter Number of iterations. - * @param trainingCost K-means cost (sum of squared distances to the nearest centroid for all - * points in the training dataset). This is equivalent to sklearn's inertia. - */ -@Since("2.0.0") -class KMeansSummary private[clustering] ( - predictions: DataFrame, - predictionCol: String, - featuresCol: String, - k: Int, - numIter: Int, - @Since("2.4.0") val trainingCost: Double) - extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/feature/PCA.scala deleted file mode 100644 index 8ca5a9b5d..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/feature/PCA.scala +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.feature - -import org.apache.hadoop.fs.Path - -import org.apache.spark.annotation.Since -import org.apache.spark.ml._ -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util._ -import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} -import org.apache.spark.sql._ -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.VersionUtils.majorVersion - -/** - * Params for [[PCA]] and [[PCAModel]]. - */ -private[feature] trait PCAParams extends Params with HasInputCol with HasOutputCol { - - /** - * The number of principal components. - * @group param - */ - final val k: IntParam = new IntParam(this, "k", "the number of principal components (> 0)", - ParamValidators.gt(0)) - - /** @group getParam */ - def getK: Int = $(k) - - /** Validates and transforms the input schema. */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - SchemaUtils.updateAttributeGroupSize(schema, $(outputCol), $(k)) - } -} - -/** - * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` - * principal components. - */ -@Since("1.5.0") -class PCA @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("pca")) - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** - * Computes a [[PCAModel]] that contains the principal components of the input vectors. - */ - @Since("2.0.0") - override def fit(dataset: Dataset[_]): PCAModel = { - transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd - val inputVectors = input.map { - case Row(v: Vector) => v - } - - val numFeatures = inputVectors.first().size - require($(k) <= numFeatures, - s"source vector size $numFeatures must be no less than k=$k") - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - - // Call oneDAL Correlation PCA implementation when numFeatures < 65535 and fall back otherwise - val parentModel = if (numFeatures < 65535 && Utils.isOAPEnabled() && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val pca = new PCADALImpl(k = $(k), executor_num, executor_cores) - val pcaModel = pca.train(inputVectors) - pcaModel - } else { - val inputOldVectors = inputVectors.map { - case v: Vector => OldVectors.fromML(v) - } - val pca = new feature.PCA(k = $(k)) - val pcaModel = pca.fit(inputOldVectors) - pcaModel - } - copyValues(new PCAModel(uid, parentModel.pc.asML, parentModel.explainedVariance.asML) - .setParent(this)) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCA = defaultCopy(extra) -} - -@Since("1.6.0") -object PCA extends DefaultParamsReadable[PCA] { - - @Since("1.6.0") - override def load(path: String): PCA = super.load(path) -} - -/** - * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. - * - * @param pc A principal components Matrix. Each column is one principal component. - * @param explainedVariance A vector of proportions of variance explained by - * each principal component. - */ -@Since("1.5.0") -class PCAModel private[ml] ( - @Since("1.5.0") override val uid: String, - @Since("2.0.0") val pc: DenseMatrix, - @Since("2.0.0") val explainedVariance: DenseVector) - extends Model[PCAModel] with PCAParams with MLWritable { - - import PCAModel._ - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** - * Transform a vector by computed Principal Components. - * - * @note Vectors to be transformed must be the same length as the source vectors given - * to `PCA.fit()`. - */ - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val transposed = pc.transpose - val transformer = udf { vector: Vector => transposed.multiply(vector) } - dataset.withColumn($(outputCol), transformer(col($(inputCol))), - outputSchema($(outputCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(outputCol).nonEmpty) { - outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, - $(outputCol), $(k)) - } - outputSchema - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCAModel = { - val copied = new PCAModel(uid, pc, explainedVariance) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new PCAModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"PCAModel: uid=$uid, k=${$(k)}" - } -} - -@Since("1.6.0") -object PCAModel extends MLReadable[PCAModel] { - - private[PCAModel] class PCAModelWriter(instance: PCAModel) extends MLWriter { - - private case class Data(pc: DenseMatrix, explainedVariance: DenseVector) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.pc, instance.explainedVariance) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class PCAModelReader extends MLReader[PCAModel] { - - private val className = classOf[PCAModel].getName - - /** - * Loads a [[PCAModel]] from data located at the input path. Note that the model includes an - * `explainedVariance` member that is not recorded by Spark 1.6 and earlier. A model - * can be loaded from such older data but will have an empty vector for - * `explainedVariance`. - * - * @param path path to serialized model data - * @return a [[PCAModel]] - */ - override def load(path: String): PCAModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val model = if (majorVersion(metadata.sparkVersion) >= 2) { - val Row(pc: DenseMatrix, explainedVariance: DenseVector) = - sparkSession.read.parquet(dataPath) - .select("pc", "explainedVariance") - .head() - new PCAModel(metadata.uid, pc, explainedVariance) - } else { - // pc field is the old matrix format in Spark <= 1.6 - // explainedVariance field is not present in Spark <= 1.6 - val Row(pc: OldDenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() - new PCAModel(metadata.uid, pc.asML, new DenseVector(Array.emptyDoubleArray)) - } - metadata.getAndSetParams(model) - model - } - } - - @Since("1.6.0") - override def read: MLReader[PCAModel] = new PCAModelReader - - @Since("1.6.0") - override def load(path: String): PCAModel = super.load(path) -} diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/regression/LinearRegression.scala deleted file mode 100644 index 6e5a4ee95..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ /dev/null @@ -1,1065 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.regression - -import scala.collection.mutable - -import breeze.linalg.{DenseVector => BDV} -import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, LBFGSB => BreezeLBFGSB, OWLQN => BreezeOWLQN} -import breeze.stats.distributions.StudentsT -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkException -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PipelineStage, PredictorParams} -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{Vector, Vectors} -import org.apache.spark.ml.linalg.BLAS._ -import org.apache.spark.ml.optim.WeightedLeastSquares -import org.apache.spark.ml.optim.aggregator.{HuberAggregator, LeastSquaresAggregator} -import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} -import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.stat._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.regression.{LinearRegressionModel => OldLinearRegressionModel} -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, DoubleType, StructType} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorMinorVersion - -/** - * Params for linear regression. - */ -private[regression] trait LinearRegressionParams extends PredictorParams - with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol - with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver - with HasAggregationDepth with HasLoss { - - import LinearRegression._ - - /** - * The solver algorithm for optimization. - * Supported options: "l-bfgs", "normal" and "auto". - * Default: "auto" - * - * @group param - */ - @Since("1.6.0") - final override val solver: Param[String] = new Param[String](this, "solver", - "The solver algorithm for optimization. Supported options: " + - s"${supportedSolvers.mkString(", ")}. (Default auto)", - ParamValidators.inArray[String](supportedSolvers)) - - /** - * The loss function to be optimized. - * Supported options: "squaredError" and "huber". - * Default: "squaredError" - * - * @group param - */ - @Since("2.3.0") - final override val loss: Param[String] = new Param[String](this, "loss", "The loss function to" + - s" be optimized. Supported options: ${supportedLosses.mkString(", ")}. (Default squaredError)", - ParamValidators.inArray[String](supportedLosses)) - - /** - * The shape parameter to control the amount of robustness. Must be > 1.0. - * At larger values of epsilon, the huber criterion becomes more similar to least squares - * regression; for small values of epsilon, the criterion is more similar to L1 regression. - * Default is 1.35 to get as much robustness as possible while retaining - * 95% statistical efficiency for normally distributed data. It matches sklearn - * HuberRegressor and is "M" from - * A robust hybrid of lasso and ridge regression. - * Only valid when "loss" is "huber". - * - * @group expertParam - */ - @Since("2.3.0") - final val epsilon = new DoubleParam(this, "epsilon", "The shape parameter to control the " + - "amount of robustness. Must be > 1.0.", ParamValidators.gt(1.0)) - - /** @group getExpertParam */ - @Since("2.3.0") - def getEpsilon: Double = $(epsilon) - - override protected def validateAndTransformSchema( - schema: StructType, - fitting: Boolean, - featuresDataType: DataType): StructType = { - if (fitting) { - if ($(loss) == Huber) { - require($(solver)!= Normal, "LinearRegression with huber loss doesn't support " + - "normal solver, please change solver to auto or l-bfgs.") - require($(elasticNetParam) == 0.0, "LinearRegression with huber loss only supports " + - s"L2 regularization, but got elasticNetParam = $getElasticNetParam.") - } - } - super.validateAndTransformSchema(schema, fitting, featuresDataType) - } -} - -/** - * Linear regression. - * - * The learning objective is to minimize the specified loss function, with regularization. - * This supports two kinds of loss: - * - squaredError (a.k.a squared loss) - * - huber (a hybrid of squared error for relatively small errors and absolute error for - * relatively large ones, and we estimate the scale parameter from training data) - * - * This supports multiple types of regularization: - * - none (a.k.a. ordinary least squares) - * - L2 (ridge regression) - * - L1 (Lasso) - * - L2 + L1 (elastic net) - * - * The squared error objective function is: - * - *
- * $$ - * \begin{align} - * \min_{w}\frac{1}{2n}{\sum_{i=1}^n(X_{i}w - y_{i})^{2} + - * \lambda\left[\frac{1-\alpha}{2}{||w||_{2}}^{2} + \alpha{||w||_{1}}\right]} - * \end{align} - * $$ - *
- * - * The huber objective function is: - * - *
- * $$ - * \begin{align} - * \min_{w, \sigma}\frac{1}{2n}{\sum_{i=1}^n\left(\sigma + - * H_m\left(\frac{X_{i}w - y_{i}}{\sigma}\right)\sigma\right) + \frac{1}{2}\lambda {||w||_2}^2} - * \end{align} - * $$ - *
- * - * where - * - *
- * $$ - * \begin{align} - * H_m(z) = \begin{cases} - * z^2, & \text {if } |z| < \epsilon, \\ - * 2\epsilon|z| - \epsilon^2, & \text{otherwise} - * \end{cases} - * \end{align} - * $$ - *
- * - * Note: Fitting with huber loss only supports none and L2 regularization. - */ -@Since("1.3.0") -class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) - extends Regressor[Vector, LinearRegression, LinearRegressionModel] - with LinearRegressionParams with DefaultParamsWritable with Logging { - - import LinearRegression._ - - @Since("1.4.0") - def this() = this(Identifiable.randomUID("linReg")) - - /** - * Set the regularization parameter. - * Default is 0.0. - * - * @group setParam - */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) - - /** - * Set if we should fit the intercept. - * Default is true. - * - * @group setParam - */ - @Since("1.5.0") - def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) - - /** - * Whether to standardize the training features before fitting the model. - * The coefficients of models will be always returned on the original scale, - * so it will be transparent for users. - * Default is true. - * - * @note With/without standardization, the models should be always converged - * to the same solution when no regularization is applied. In R's GLMNET package, - * the default behavior is true as well. - * - * @group setParam - */ - @Since("1.5.0") - def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) - - /** - * Set the ElasticNet mixing parameter. - * For alpha = 0, the penalty is an L2 penalty. - * For alpha = 1, it is an L1 penalty. - * For alpha in (0,1), the penalty is a combination of L1 and L2. - * Default is 0.0 which is an L2 penalty. - * - * Note: Fitting with huber loss only supports None and L2 regularization, - * so throws exception if this param is non-zero value. - * - * @group setParam - */ - @Since("1.4.0") - def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - setDefault(elasticNetParam -> 0.0) - - /** - * Set the maximum number of iterations. - * Default is 100. - * - * @group setParam - */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) - - /** - * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. - * Default is 1E-6. - * - * @group setParam - */ - @Since("1.4.0") - def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) - - /** - * Whether to over-/under-sample training instances according to the given weights in weightCol. - * If not set or empty, all instances are treated equally (weight 1.0). - * Default is not set, so all instances have weight one. - * - * @group setParam - */ - @Since("1.6.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - /** - * Set the solver algorithm used for optimization. - * In case of linear regression, this can be "l-bfgs", "normal" and "auto". - * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton - * optimization method. - * - "normal" denotes using Normal Equation as an analytical solution to the linear regression - * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. - * - "auto" (default) means that the solver algorithm is selected automatically. - * The Normal Equations solver will be used when possible, but this will automatically fall - * back to iterative optimization methods when needed. - * - * Note: Fitting with huber loss doesn't support normal solver, - * so throws exception if this param was set with "normal". - * @group setParam - */ - @Since("1.6.0") - def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> Auto) - - /** - * Suggested depth for treeAggregate (greater than or equal to 2). - * If the dimensions of features or the number of partitions are large, - * this param could be adjusted to a larger size. - * Default is 2. - * - * @group expertSetParam - */ - @Since("2.1.0") - def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) - - /** - * Sets the value of param [[loss]]. - * Default is "squaredError". - * - * @group setParam - */ - @Since("2.3.0") - def setLoss(value: String): this.type = set(loss, value) - setDefault(loss -> SquaredError) - - /** - * Sets the value of param [[epsilon]]. - * Default is 1.35. - * - * @group setExpertParam - */ - @Since("2.3.0") - def setEpsilon(value: Double): this.type = set(epsilon, value) - setDefault(epsilon -> 1.35) - - override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => - // Extract the number of features before deciding optimization solver. - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) - - val instances = extractInstances(dataset) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, solver, tol, - elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth, loss, - epsilon) - instr.logNumFeatures(numFeatures) - - if ($(loss) == SquaredError && (($(solver) == Auto && - numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == Normal)) { - // oneDAL only support simple linear regression and ridge regression - val paramSupported = ($(regParam) == 0) || ($(regParam) != 0 && $(elasticNetParam) == 0) - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - if (paramSupported && Utils.isOAPEnabled && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - logInfo(s"LinearRegressionDAL fit using $executor_num Executors") - - val optimizer = new LinearRegressionDALImpl($(fitIntercept), $(regParam), - elasticNetParam = $(elasticNetParam), $(standardization), true, - executor_num, executor_cores) - - // Return same model as WeightedLeastSquaresModel - val model = optimizer.train(dataset, Some(instr)) - - val lrModel = copyValues( - new LinearRegressionModel(uid, model.coefficients, model.intercept)) - - val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - summaryModel, - model.diagInvAtWA.toArray, - model.objectiveHistory) - - return lrModel.setSummary(Some(trainingSummary)) - } else { - // For low dimensional data, WeightedLeastSquares is more efficient since the - // training algorithm only requires one pass through the data. (SPARK-10668) - - val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), - elasticNetParam = $(elasticNetParam), $(standardization), true, - solverType = WeightedLeastSquares.Auto, maxIter = $(maxIter), tol = $(tol)) - val model = optimizer.fit(instances, instr = OptionalInstrumentation.create(instr)) - // When it is trained by WeightedLeastSquares, training summary does not - // attach returned model. - val lrModel = copyValues(new LinearRegressionModel(uid, model.coefficients, model.intercept)) - val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - summaryModel, - model.diagInvAtWA.toArray, - model.objectiveHistory) - - return lrModel.setSummary(Some(trainingSummary)) - } - } - - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - - val (featuresSummarizer, ySummarizer) = instances.treeAggregate( - (Summarizer.createSummarizerBuffer("mean", "std"), - Summarizer.createSummarizerBuffer("mean", "std", "count")))( - seqOp = (c: (SummarizerBuffer, SummarizerBuffer), instance: Instance) => - (c._1.add(instance.features, instance.weight), - c._2.add(Vectors.dense(instance.label), instance.weight)), - combOp = (c1: (SummarizerBuffer, SummarizerBuffer), - c2: (SummarizerBuffer, SummarizerBuffer)) => - (c1._1.merge(c2._1), c1._2.merge(c2._2)), - depth = $(aggregationDepth) - ) - - val yMean = ySummarizer.mean(0) - val rawYStd = ySummarizer.std(0) - - instr.logNumExamples(ySummarizer.count) - instr.logNamedValue(Instrumentation.loggerTags.meanOfLabels, yMean) - instr.logNamedValue(Instrumentation.loggerTags.varianceOfLabels, rawYStd) - instr.logSumOfWeights(featuresSummarizer.weightSum) - - if (rawYStd == 0.0) { - if ($(fitIntercept) || yMean == 0.0) { - // If the rawYStd==0 and fitIntercept==true, then the intercept is yMean with - // zero coefficient; as a result, training is not needed. - // Also, if yMean==0 and rawYStd==0, all the coefficients are zero regardless of - // the fitIntercept. - if (yMean == 0.0) { - instr.logWarning(s"Mean and standard deviation of the label are zero, so the " + - s"coefficients and the intercept will all be zero; as a result, training is not " + - s"needed.") - } else { - instr.logWarning(s"The standard deviation of the label is zero, so the coefficients " + - s"will be zeros and the intercept will be the mean of the label; as a result, " + - s"training is not needed.") - } - if (handlePersistence) instances.unpersist() - val coefficients = Vectors.sparse(numFeatures, Seq.empty) - val intercept = yMean - - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - model, - Array(0D), - Array(0D)) - - return model.setSummary(Some(trainingSummary)) - } else { - require($(regParam) == 0.0, "The standard deviation of the label is zero. " + - "Model cannot be regularized.") - instr.logWarning(s"The standard deviation of the label is zero. " + - "Consider setting fitIntercept=true.") - } - } - - // if y is constant (rawYStd is zero), then y cannot be scaled. In this case - // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. - val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) - val featuresMean = featuresSummarizer.mean.toArray - val featuresStd = featuresSummarizer.std.toArray - val bcFeaturesMean = instances.context.broadcast(featuresMean) - val bcFeaturesStd = instances.context.broadcast(featuresStd) - - if (!$(fitIntercept) && (0 until numFeatures).exists { i => - featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { - instr.logWarning("Fitting LinearRegressionModel without intercept on dataset with " + - "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + - "columns. This behavior is the same as R glmnet but different from LIBSVM.") - } - - // Since we implicitly do the feature scaling when we compute the cost function - // to improve the convergence, the effective regParam will be changed. - val effectiveRegParam = $(loss) match { - case SquaredError => $(regParam) / yStd - case Huber => $(regParam) - } - val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam - val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam - - val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 - val regularization = if (effectiveL2RegParam != 0.0) { - val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures - Some(new L2Regularization(effectiveL2RegParam, shouldApply, - if ($(standardization)) None else Some(getFeaturesStd))) - } else { - None - } - - val costFun = $(loss) match { - case SquaredError => - val getAggregatorFunc = new LeastSquaresAggregator(yStd, yMean, $(fitIntercept), - bcFeaturesStd, bcFeaturesMean)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) - case Huber => - val getAggregatorFunc = new HuberAggregator($(fitIntercept), $(epsilon), bcFeaturesStd)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) - } - - val optimizer = $(loss) match { - case SquaredError => - if ($(elasticNetParam) == 0.0 || effectiveRegParam == 0.0) { - new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) - } else { - val standardizationParam = $(standardization) - def effectiveL1RegFun = (index: Int) => { - if (standardizationParam) { - effectiveL1RegParam - } else { - // If `standardization` is false, we still standardize the data - // to improve the rate of convergence; as a result, we have to - // perform this reverse standardization by penalizing each component - // differently to get effectively the same objective function when - // the training dataset is not standardized. - if (featuresStd(index) != 0.0) effectiveL1RegParam / featuresStd(index) else 0.0 - } - } - new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, effectiveL1RegFun, $(tol)) - } - case Huber => - val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 - val lowerBounds = BDV[Double](Array.fill(dim)(Double.MinValue)) - // Optimize huber loss in space "\sigma > 0" - lowerBounds(dim - 1) = Double.MinPositiveValue - val upperBounds = BDV[Double](Array.fill(dim)(Double.MaxValue)) - new BreezeLBFGSB(lowerBounds, upperBounds, $(maxIter), 10, $(tol)) - } - - val initialValues = $(loss) match { - case SquaredError => - Vectors.zeros(numFeatures) - case Huber => - val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 - Vectors.dense(Array.fill(dim)(1.0)) - } - - val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialValues.asBreeze.toDenseVector) - - val (coefficients, intercept, scale, objectiveHistory) = { - /* - Note that in Linear Regression, the objective history (loss + regularization) returned - from optimizer is computed in the scaled space given by the following formula. -
- $$ - L &= 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 - + regTerms \\ - $$ -
- */ - val arrayBuilder = mutable.ArrayBuilder.make[Double] - var state: optimizer.State = null - while (states.hasNext) { - state = states.next() - arrayBuilder += state.adjustedValue - } - if (state == null) { - val msg = s"${optimizer.getClass.getName} failed." - instr.logError(msg) - throw new SparkException(msg) - } - - bcFeaturesMean.destroy() - bcFeaturesStd.destroy() - - val parameters = state.x.toArray.clone() - - /* - The coefficients are trained in the scaled space; we're converting them back to - the original space. - */ - val rawCoefficients: Array[Double] = $(loss) match { - case SquaredError => parameters - case Huber => parameters.slice(0, numFeatures) - } - - var i = 0 - val len = rawCoefficients.length - val multiplier = $(loss) match { - case SquaredError => yStd - case Huber => 1.0 - } - while (i < len) { - rawCoefficients(i) *= { if (featuresStd(i) != 0.0) multiplier / featuresStd(i) else 0.0 } - i += 1 - } - - val interceptValue: Double = if ($(fitIntercept)) { - $(loss) match { - case SquaredError => - /* - The intercept of squared error in R's GLMNET is computed using closed form - after the coefficients are converged. See the following discussion for detail. - http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet - */ - yMean - dot(Vectors.dense(rawCoefficients), Vectors.dense(featuresMean)) - case Huber => parameters(numFeatures) - } - } else { - 0.0 - } - - val scaleValue: Double = $(loss) match { - case SquaredError => 1.0 - case Huber => parameters.last - } - - (Vectors.dense(rawCoefficients).compressed, interceptValue, scaleValue, arrayBuilder.result()) - } - - if (handlePersistence) instances.unpersist() - - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept, scale)) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - model, - Array(0D), - objectiveHistory) - - model.setSummary(Some(trainingSummary)) - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra) -} - -@Since("1.6.0") -object LinearRegression extends DefaultParamsReadable[LinearRegression] { - - @Since("1.6.0") - override def load(path: String): LinearRegression = super.load(path) - - /** - * When using `LinearRegression.solver` == "normal", the solver must limit the number of - * features to at most this number. The entire covariance matrix X^T^X will be collected - * to the driver. This limit helps prevent memory overflow errors. - */ - @Since("2.1.0") - val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES - - /** String name for "auto". */ - private[regression] val Auto = "auto" - - /** String name for "normal". */ - private[regression] val Normal = "normal" - - /** String name for "l-bfgs". */ - private[regression] val LBFGS = "l-bfgs" - - /** Set of solvers that LinearRegression supports. */ - private[regression] val supportedSolvers = Array(Auto, Normal, LBFGS) - - /** String name for "squaredError". */ - private[regression] val SquaredError = "squaredError" - - /** String name for "huber". */ - private[regression] val Huber = "huber" - - /** Set of loss function names that LinearRegression supports. */ - private[regression] val supportedLosses = Array(SquaredError, Huber) -} - -/** - * Model produced by [[LinearRegression]]. - */ -@Since("1.3.0") -class LinearRegressionModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("2.0.0") val coefficients: Vector, - @Since("1.3.0") val intercept: Double, - @Since("2.3.0") val scale: Double) - extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams with GeneralMLWritable - with HasTrainingSummary[LinearRegressionTrainingSummary] { - - private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = - this(uid, coefficients, intercept, 1.0) - - override val numFeatures: Int = coefficients.size - - /** - * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("1.5.0") - override def summary: LinearRegressionTrainingSummary = super.summary - - /** - * Evaluates the model on a test dataset. - * - * @param dataset Test dataset to evaluate model on. - */ - @Since("2.0.0") - def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() - new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, - $(labelCol), $(featuresCol), summaryModel, Array(0D)) - } - - /** - * If the prediction column is set returns the current model and prediction column, - * otherwise generates a new column and sets it as the prediction column on a new copy - * of the current model. - */ - private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { - $(predictionCol) match { - case "" => - val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString - (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) - case p => (this, p) - } - } - - - override def predict(features: Vector): Double = { - dot(features, coefficients) + intercept - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegressionModel = { - val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) - newModel.setSummary(trainingSummary).setParent(parent) - } - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - * This also does not save the [[parent]] currently. - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" - } -} - -/** A writer for LinearRegression that handles the "internal" (or default) format */ -private class InternalLinearRegressionModelWriter - extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" - - private case class Data(intercept: Double, coefficients: Vector, scale: Double) - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[LinearRegressionModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: intercept, coefficients, scale - val data = Data(instance.intercept, instance.coefficients, instance.scale) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for LinearRegression that handles the "pmml" format */ -private class PMMLLinearRegressionModelWriter - extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - - override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" - - private case class Data(intercept: Double, coefficients: Vector) - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val sc = sparkSession.sparkContext - // Construct the MLLib model which knows how to write to PMML. - val instance = stage.asInstanceOf[LinearRegressionModel] - val oldModel = new OldLinearRegressionModel(instance.coefficients, instance.intercept) - // Save PMML - oldModel.toPMML(sc, path) - } -} - -@Since("1.6.0") -object LinearRegressionModel extends MLReadable[LinearRegressionModel] { - - @Since("1.6.0") - override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader - - @Since("1.6.0") - override def load(path: String): LinearRegressionModel = super.load(path) - - private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[LinearRegressionModel].getName - - override def load(path: String): LinearRegressionModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.format("parquet").load(dataPath) - val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) - val model = if (majorVersion < 2 || (majorVersion == 2 && minorVersion <= 2)) { - // Spark 2.2 and before - val Row(intercept: Double, coefficients: Vector) = - MLUtils.convertVectorColumnsToML(data, "coefficients") - .select("intercept", "coefficients") - .head() - new LinearRegressionModel(metadata.uid, coefficients, intercept) - } else { - // Spark 2.3 and later - val Row(intercept: Double, coefficients: Vector, scale: Double) = - data.select("intercept", "coefficients", "scale").head() - new LinearRegressionModel(metadata.uid, coefficients, intercept, scale) - } - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Linear regression training results. Currently, the training summary ignores the - * training weights except for the objective trace. - * - * @param predictions predictions output by the model's `transform` method. - * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. - */ -@Since("1.5.0") -class LinearRegressionTrainingSummary private[regression] ( - predictions: DataFrame, - predictionCol: String, - labelCol: String, - featuresCol: String, - model: LinearRegressionModel, - diagInvAtWA: Array[Double], - val objectiveHistory: Array[Double]) - extends LinearRegressionSummary( - predictions, - predictionCol, - labelCol, - featuresCol, - model, - diagInvAtWA) { - - /** - * Number of training iterations until termination - * - * This value is only available when using the "l-bfgs" solver. - * - * @see `LinearRegression.solver` - */ - @Since("1.5.0") - val totalIterations = objectiveHistory.length - -} - -/** - * Linear regression results evaluated on a dataset. - * - * @param predictions predictions output by the model's `transform` method. - * @param predictionCol Field in "predictions" which gives the predicted value of the label at - * each instance. - * @param labelCol Field in "predictions" which gives the true label of each instance. - * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. - */ -@Since("1.5.0") -class LinearRegressionSummary private[regression] ( - @transient val predictions: DataFrame, - val predictionCol: String, - val labelCol: String, - val featuresCol: String, - private val privateModel: LinearRegressionModel, - private val diagInvAtWA: Array[Double]) extends Serializable { - - @transient private val metrics = new RegressionMetrics( - predictions - .select(col(predictionCol), col(labelCol).cast(DoubleType)) - .rdd - .map { case Row(pred: Double, label: Double) => (pred, label) }, - !privateModel.getFitIntercept) - - /** - * Returns the explained variance regression score. - * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) - * Reference: - * Wikipedia explain variation - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val explainedVariance: Double = metrics.explainedVariance - - /** - * Returns the mean absolute error, which is a risk function corresponding to the - * expected value of the absolute error loss or l1-norm loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val meanAbsoluteError: Double = metrics.meanAbsoluteError - - /** - * Returns the mean squared error, which is a risk function corresponding to the - * expected value of the squared error loss or quadratic loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val meanSquaredError: Double = metrics.meanSquaredError - - /** - * Returns the root mean squared error, which is defined as the square root of - * the mean squared error. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val rootMeanSquaredError: Double = metrics.rootMeanSquaredError - - /** - * Returns R^2^, the coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val r2: Double = metrics.r2 - - /** - * Returns Adjusted R^2^, the adjusted coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("2.3.0") - val r2adj: Double = { - val interceptDOF = if (privateModel.getFitIntercept) 1 else 0 - 1 - (1 - r2) * (numInstances - interceptDOF) / - (numInstances - privateModel.coefficients.size - interceptDOF) - } - - /** Residuals (label - predicted value) */ - @Since("1.5.0") - @transient lazy val residuals: DataFrame = { - val t = udf { (pred: Double, label: Double) => label - pred } - predictions.select(t(col(predictionCol), col(labelCol)).as("residuals")) - } - - /** Number of instances in DataFrame predictions */ - lazy val numInstances: Long = predictions.count() - - /** Degrees of freedom */ - @Since("2.2.0") - val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { - numInstances - privateModel.coefficients.size - 1 - } else { - numInstances - privateModel.coefficients.size - } - - /** - * The weighted residuals, the usual residuals rescaled by - * the square root of the instance weights. - */ - lazy val devianceResiduals: Array[Double] = { - val weighted = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - lit(1.0) - } else { - sqrt(col(privateModel.getWeightCol)) - } - val dr = predictions - .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) - .multiply(weighted).as("weightedResiduals")) - .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) - .first() - Array(dr.getDouble(0), dr.getDouble(1)) - } - - /** - * Standard error of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val coefficientStandardErrors: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No Std. Error of coefficients available for this LinearRegressionModel") - } else { - val rss = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - meanSquaredError * numInstances - } else { - val t = udf { (pred: Double, label: Double, weight: Double) => - math.pow(label - pred, 2.0) * weight } - predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), - col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) - } - val sigma2 = rss / degreesOfFreedom - diagInvAtWA.map(_ * sigma2).map(math.sqrt) - } - } - - /** - * T-statistic of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val tValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No t-statistic available for this LinearRegressionModel") - } else { - val estimate = if (privateModel.getFitIntercept) { - Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) - } else { - privateModel.coefficients.toArray - } - estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } - } - } - - /** - * Two-sided p-value of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val pValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No p-value available for this LinearRegressionModel") - } else { - tValues.map { x => 2.0 * (1.0 - StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) } - } - } - -} - diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/stat/Correlation.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/stat/Correlation.scala deleted file mode 100644 index 5b411011a..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/ml/stat/Correlation.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.stat - -import org.apache.spark.annotation.{Experimental, Since} -import org.apache.spark.ml.linalg.{SQLDataTypes, Vector} -import org.apache.spark.ml.util.Utils -import org.apache.spark.mllib.linalg.{Vectors => OldVectors} -import org.apache.spark.mllib.stat.{Statistics => OldStatistics} -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, Row} -import org.apache.spark.storage.StorageLevel - -import scala.collection.JavaConverters._ - -/** - * API for correlation functions in MLlib, compatible with DataFrames and Datasets. - * - * The functions in this package generalize the functions in [[org.apache.spark.sql.Dataset#stat]] - * to spark.ml's Vector types. - */ -@Since("2.2.0") -@Experimental -object Correlation { - - /** - * :: Experimental :: - * Compute the correlation matrix for the input Dataset of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param dataset A dataset or a dataframe - * @param column The name of the column of vectors for which the correlation coefficient needs - * to be computed. This must be a column of the dataset, and it must contain - * Vector objects. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return A dataframe that contains the correlation matrix of the column of vectors. This - * dataframe contains a single row and a single column of name - * '$METHODNAME($COLUMN)'. - * @throws IllegalArgumentException if the column is not a valid column in the dataset, or if - * the content of this column is not of type Vector. - * - * Here is how to access the correlation coefficient: - * {{{ - * val data: Dataset[Vector] = ... - * val Row(coeff: Matrix) = Correlation.corr(data, "value").head - * // coeff now contains the Pearson correlation matrix. - * }}} - * - * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input Dataset before calling corr with `method = "spearman"` - * to avoid recomputing the common lineage. - */ - @Since("2.2.0") - def corr(dataset: Dataset[_], column: String, method: String): DataFrame = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - if (Utils.isOAPEnabled() && isPlatformSupported && method == "pearson") { - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - if (handlePersistence) { - dataset.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = dataset.select(column).rdd.map { - case Row(v: Vector) => v - } - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val matrix = new CorrelationDALImpl(executor_num, executor_cores) - .computeCorrelationMatrix(rdd) - val name = s"$method($column)" - val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) - val dataframe = dataset.sparkSession.createDataFrame(Seq(Row(matrix)).asJava, schema) - if (handlePersistence) { - dataset.unpersist() - } - dataframe - } else { - val rdd = dataset.select(column).rdd.map { - case Row(v: Vector) => OldVectors.fromML(v) - } - val oldM = OldStatistics.corr(rdd, method) - val name = s"$method($column)" - val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) - dataset.sparkSession.createDataFrame(Seq(Row(oldM.asML)).asJava, schema) - } - } - - /** - * Compute the Pearson correlation matrix for the input Dataset of Vectors. - */ - @Since("2.2.0") - def corr(dataset: Dataset[_], column: String): DataFrame = { - corr(dataset, column, "pearson") - } -} diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/clustering/KMeans.scala deleted file mode 100644 index 3a417d3e3..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ /dev/null @@ -1,523 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.clustering - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.annotation.Since -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.Logging -import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS.axpy -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils -import org.apache.spark.util.random.XORShiftRandom - -/** - * K-means clustering with a k-means++ like initialization mode - * (the k-means|| algorithm by Bahmani et al). - * - * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given - * to it should be cached by the user. - */ -@Since("0.8.0") -class KMeans private ( - private var k: Int, - private var maxIterations: Int, - private var initializationMode: String, - private var initializationSteps: Int, - private var epsilon: Double, - private var seed: Long, - private var distanceMeasure: String) extends Serializable with Logging { - - @Since("0.8.0") - private def this(k: Int, maxIterations: Int, initializationMode: String, initializationSteps: Int, - epsilon: Double, seed: Long) = - this(k, maxIterations, initializationMode, initializationSteps, - epsilon, seed, DistanceMeasure.EUCLIDEAN) - - /** - * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, - * initializationMode: "k-means||", initializationSteps: 2, epsilon: 1e-4, seed: random, - * distanceMeasure: "euclidean"}. - */ - @Since("0.8.0") - def this() = this(2, 20, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong(), - DistanceMeasure.EUCLIDEAN) - - /** - * Number of clusters to create (k). - * - * @note It is possible for fewer than k clusters to - * be returned, for example, if there are fewer than k distinct points to cluster. - */ - @Since("1.4.0") - def getK: Int = k - - /** - * Set the number of clusters to create (k). - * - * @note It is possible for fewer than k clusters to - * be returned, for example, if there are fewer than k distinct points to cluster. Default: 2. - */ - @Since("0.8.0") - def setK(k: Int): this.type = { - require(k > 0, - s"Number of clusters must be positive but got ${k}") - this.k = k - this - } - - /** - * Maximum number of iterations allowed. - */ - @Since("1.4.0") - def getMaxIterations: Int = maxIterations - - /** - * Set maximum number of iterations allowed. Default: 20. - */ - @Since("0.8.0") - def setMaxIterations(maxIterations: Int): this.type = { - require(maxIterations >= 0, - s"Maximum of iterations must be nonnegative but got ${maxIterations}") - this.maxIterations = maxIterations - this - } - - /** - * The initialization algorithm. This can be either "random" or "k-means||". - */ - @Since("1.4.0") - def getInitializationMode: String = initializationMode - - /** - * Set the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - */ - @Since("0.8.0") - def setInitializationMode(initializationMode: String): this.type = { - KMeans.validateInitMode(initializationMode) - this.initializationMode = initializationMode - this - } - - /** - * Number of steps for the k-means|| initialization mode - */ - @Since("1.4.0") - def getInitializationSteps: Int = initializationSteps - - /** - * Set the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Default: 2. - */ - @Since("0.8.0") - def setInitializationSteps(initializationSteps: Int): this.type = { - require(initializationSteps > 0, - s"Number of initialization steps must be positive but got ${initializationSteps}") - this.initializationSteps = initializationSteps - this - } - - /** - * The distance threshold within which we've consider centers to have converged. - */ - @Since("1.4.0") - def getEpsilon: Double = epsilon - - /** - * Set the distance threshold within which we've consider centers to have converged. - * If all centers move less than this Euclidean distance, we stop iterating one run. - */ - @Since("0.8.0") - def setEpsilon(epsilon: Double): this.type = { - require(epsilon >= 0, - s"Distance threshold must be nonnegative but got ${epsilon}") - this.epsilon = epsilon - this - } - - /** - * The random seed for cluster initialization. - */ - @Since("1.4.0") - def getSeed: Long = seed - - /** - * Set the random seed for cluster initialization. - */ - @Since("1.4.0") - def setSeed(seed: Long): this.type = { - this.seed = seed - this - } - - /** - * The distance suite used by the algorithm. - */ - @Since("2.4.0") - def getDistanceMeasure: String = distanceMeasure - - /** - * Set the distance suite used by the algorithm. - */ - @Since("2.4.0") - def setDistanceMeasure(distanceMeasure: String): this.type = { - DistanceMeasure.validateDistanceMeasure(distanceMeasure) - this.distanceMeasure = distanceMeasure - this - } - - // Initial cluster centers can be provided as a KMeansModel object rather than using the - // random or k-means|| initializationMode - private var initialModel: Option[KMeansModel] = None - - /** - * Set the initial starting point, bypassing the random initialization or k-means|| - * The condition model.k == this.k must be met, failure results - * in an IllegalArgumentException. - */ - @Since("1.4.0") - def setInitialModel(model: KMeansModel): this.type = { - require(model.k == k, "mismatched cluster count") - initialModel = Some(model) - this - } - - /** - * Train a K-means model on the given set of points; `data` should be cached for high - * performance, because this is an iterative algorithm. - */ - @Since("0.8.0") - def run(data: RDD[Vector]): KMeansModel = { - val instances: RDD[(Vector, Double)] = data.map { - case (point) => (point, 1.0) - } - runWithWeight(instances, None) - } - - private[spark] def runWithWeight( - data: RDD[(Vector, Double)], - instr: Option[Instrumentation]): KMeansModel = { - - // Compute squared norms and cache them. - val norms = data.map { case (v, _) => - Vectors.norm(v, 2.0) - } - - val zippedData = data.zip(norms).map { case ((v, w), norm) => - new VectorWithNorm(v, norm, w) - } - - if (data.getStorageLevel == StorageLevel.NONE) { - zippedData.persist(StorageLevel.MEMORY_AND_DISK) - } - val model = runAlgorithmWithWeight(zippedData, instr) - zippedData.unpersist() - - model - } - - /** - * Implementation of K-Means algorithm. - */ - private def runAlgorithmWithWeight( - data: RDD[VectorWithNorm], - instr: Option[Instrumentation]): KMeansModel = { - - val sc = data.sparkContext - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString(this.distanceMeasure) - - val centers = initialModel match { - case Some(kMeansCenters) => - kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) - case None => - if (initializationMode == KMeans.RANDOM) { - initRandom(data) - } else { - initKMeansParallel(data, distanceMeasureInstance) - } - } - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - logInfo(f"Initialization with $initializationMode took $initTimeInSeconds%.3f seconds.") - - var converged = false - var cost = 0.0 - var iteration = 0 - - val iterationStartTime = System.nanoTime() - - instr.foreach(_.logNumFeatures(centers.head.vector.size)) - - // Execute iterations of Lloyd's algorithm until converged - while (iteration < maxIterations && !converged) { - val costAccum = sc.doubleAccumulator - val bcCenters = sc.broadcast(centers) - - // Find the new centers - val collected = data.mapPartitions { points => - val thisCenters = bcCenters.value - val dims = thisCenters.head.vector.size - - val sums = Array.fill(thisCenters.length)(Vectors.zeros(dims)) - - // clusterWeightSum is needed to calculate cluster center - // cluster center = - // sample1 * weight1/clusterWeightSum + sample2 * weight2/clusterWeightSum + ... - val clusterWeightSum = Array.ofDim[Double](thisCenters.length) - - points.foreach { point => - val (bestCenter, cost) = distanceMeasureInstance.findClosest(thisCenters, point) - costAccum.add(cost * point.weight) - distanceMeasureInstance.updateClusterSum(point, sums(bestCenter)) - clusterWeightSum(bestCenter) += point.weight - } - - clusterWeightSum.indices.filter(clusterWeightSum(_) > 0) - .map(j => (j, (sums(j), clusterWeightSum(j)))).iterator - }.reduceByKey { (sumweight1, sumweight2) => - axpy(1.0, sumweight2._1, sumweight1._1) - (sumweight1._1, sumweight1._2 + sumweight2._2) - }.collectAsMap() - - if (iteration == 0) { - instr.foreach(_.logNumExamples(costAccum.count)) - instr.foreach(_.logSumOfWeights(collected.values.map(_._2).sum)) - } - - val newCenters = collected.mapValues { case (sum, weightSum) => - distanceMeasureInstance.centroid(sum, weightSum) - } - - bcCenters.destroy() - - // Update the cluster centers and costs - converged = true - newCenters.foreach { case (j, newCenter) => - if (converged && - !distanceMeasureInstance.isCenterConverged(centers(j), newCenter, epsilon)) { - converged = false - } - centers(j) = newCenter - } - - cost = costAccum.value - iteration += 1 - } - - val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 - logInfo(f"Iterations took $iterationTimeInSeconds%.3f seconds.") - - if (iteration == maxIterations) { - logInfo(s"KMeans reached the max number of iterations: $maxIterations.") - } else { - logInfo(s"KMeans converged in $iteration iterations.") - } - - logInfo(s"The cost is $cost.") - - new KMeansModel(centers.map(_.vector), distanceMeasure, cost, iteration) - } - - /** - * Initialize a set of cluster centers at random. - */ - private[spark] def initRandom(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { - // Select without replacement; may still produce duplicates if the data has < k distinct - // points, so deduplicate the centroids to match the behavior of k-means|| in the same situation - data.takeSample(false, k, new XORShiftRandom(this.seed).nextInt()) - .map(_.vector).distinct.map(new VectorWithNorm(_)) - } - - /** - * Initialize a set of cluster centers using the k-means|| algorithm by Bahmani et al. - * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries - * to find dissimilar cluster centers by starting with a random center and then doing - * passes where more centers are chosen with probability proportional to their squared distance - * to the current cluster set. It results in a provable approximation to an optimal clustering. - * - * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. - */ - private[spark] def initKMeansParallel(data: RDD[VectorWithNorm], - distanceMeasureInstance: DistanceMeasure): Array[VectorWithNorm] = { - // Initialize empty centers and point costs. - var costs = data.map(_ => Double.PositiveInfinity) - - // Initialize the first center to a random point. - val seed = new XORShiftRandom(this.seed).nextInt() - val sample = data.takeSample(false, 1, seed) - // Could be empty if data is empty; fail with a better message early: - require(sample.nonEmpty, s"No samples available from $data") - - val centers = ArrayBuffer[VectorWithNorm]() - var newCenters = Seq(sample.head.toDense) - centers ++= newCenters - - // On each step, sample 2 * k points on average with probability proportional - // to their squared distance from the centers. Note that only distances between points - // and new centers are computed in each iteration. - var step = 0 - val bcNewCentersList = ArrayBuffer[Broadcast[_]]() - while (step < initializationSteps) { - val bcNewCenters = data.context.broadcast(newCenters) - bcNewCentersList += bcNewCenters - val preCosts = costs - costs = data.zip(preCosts).map { case (point, cost) => - math.min(distanceMeasureInstance.pointCost(bcNewCenters.value, point), cost) - }.persist(StorageLevel.MEMORY_AND_DISK) - val sumCosts = costs.sum() - - bcNewCenters.unpersist() - preCosts.unpersist() - - val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointCosts) => - val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - pointCosts.filter { case (_, c) => rand.nextDouble() < 2.0 * c * k / sumCosts }.map(_._1) - }.collect() - newCenters = chosen.map(_.toDense) - centers ++= newCenters - step += 1 - } - - costs.unpersist() - bcNewCentersList.foreach(_.destroy()) - - val distinctCenters = centers.map(_.vector).distinct.map(new VectorWithNorm(_)) - - if (distinctCenters.size <= k) { - distinctCenters.toArray - } else { - // Finally, we might have a set of more than k distinct candidate centers; weight each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick k of them - val bcCenters = data.context.broadcast(distinctCenters) - val countMap = data - .map(distanceMeasureInstance.findClosest(bcCenters.value, _)._1) - .countByValue() - - bcCenters.destroy() - - val myWeights = distinctCenters.indices.map(countMap.getOrElse(_, 0L).toDouble).toArray - LocalKMeans.kMeansPlusPlus(0, distinctCenters.toArray, myWeights, k, 30) - } - } -} - - -/** - * Top-level methods for calling K-means clustering. - */ -@Since("0.8.0") -object KMeans { - - // Initialization mode names - @Since("0.8.0") - val RANDOM = "random" - @Since("0.8.0") - val K_MEANS_PARALLEL = "k-means||" - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - * @param seed Random seed for cluster initialization. Default is to generate seed based - * on system time. - */ - @Since("2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - initializationMode: String, - seed: Long): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .setSeed(seed) - .run(data) - } - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - */ - @Since("2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - initializationMode: String): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .run(data) - } - - /** - * Trains a k-means model using specified parameters and the default values for unspecified. - */ - @Since("0.8.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .run(data) - } - - private[spark] def validateInitMode(initMode: String): Boolean = { - initMode match { - case KMeans.RANDOM => true - case KMeans.K_MEANS_PARALLEL => true - case _ => false - } - } -} - -/** - * A vector with its norm for fast distance computation. - */ -private[spark] class VectorWithNorm( - val vector: Vector, - val norm: Double, - val weight: Double = 1.0) extends Serializable { - - def this(vector: Vector) = this(vector, Vectors.norm(vector, 2.0)) - - def this(array: Array[Double]) = this(Vectors.dense(array)) - - /** Converts the vector to a dense vector. */ - def toDense: VectorWithNorm = new VectorWithNorm(Vectors.dense(vector.toArray), norm, weight) -} diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala deleted file mode 100644 index fa40a1000..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ /dev/null @@ -1,675 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.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.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) - - setDefault(smoothing -> 1.0, modelType -> NaiveBayes.Multinomial) -} - -// 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) - - /** - * 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) - - /** - * 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 sc = dataset.sparkSession.sparkContext - val model = if (Utils.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) - } - model - 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 spark = dataset.sparkSession - import spark.implicits._ - - val sc = spark.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"NaiveBayesDAL fit using $executor_num Executors") - - // 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) - - // 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 - } - - instr.logNumClasses(numClasses) - - val labeledPointsDS = dataset - .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) - - val model = new NaiveBayesDALImpl(uid, numClasses, - 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 - model.setOldLabels(labels) - - 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/spark-3.0.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala deleted file mode 100644 index 51ea41fe8..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ /dev/null @@ -1,494 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.clustering - -import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.Since -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.ml.util._ -import org.apache.spark.ml.{Estimator, Model, PipelineStage} -import org.apache.spark.mllib.clustering.{DistanceMeasure, VectorWithNorm, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorVersion - -import scala.collection.mutable - - -/** - * Common params for KMeans and KMeansModel - */ -private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFeaturesCol - with HasSeed with HasPredictionCol with HasTol with HasDistanceMeasure with HasWeightCol { - - /** - * The number of clusters to create (k). Must be > 1. Note that it is possible for fewer than - * k clusters to be returned, for example, if there are fewer than k distinct points to cluster. - * Default: 2. - * @group param - */ - @Since("1.5.0") - final val k = new IntParam(this, "k", "The number of clusters to create. " + - "Must be > 1.", ParamValidators.gt(1)) - - /** @group getParam */ - @Since("1.5.0") - def getK: Int = $(k) - - /** - * Param for the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - * @group expertParam - */ - @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + - "Supported options: 'random' and 'k-means||'.", - (value: String) => MLlibKMeans.validateInitMode(value)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitMode: String = $(initMode) - - /** - * Param for the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Must be > 0. Default: 2. - * @group expertParam - */ - @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + - "initialization mode. Must be > 0.", ParamValidators.gt(0)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitSteps: Int = $(initSteps) - - setDefault(k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, initSteps -> 2, - tol -> 1e-4, distanceMeasure -> DistanceMeasure.EUCLIDEAN) - - /** - * Validates and transforms the input schema. - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.validateVectorCompatibleColumn(schema, getFeaturesCol) - SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) - } -} - -/** - * Model fitted by KMeans. - * - * @param parentModel a model trained by spark.mllib.clustering.KMeans. - */ -@Since("1.5.0") -class KMeansModel private[ml] ( - @Since("1.5.0") override val uid: String, - private[clustering] val parentModel: MLlibKMeansModel) - extends Model[KMeansModel] with KMeansParams with GeneralMLWritable - with HasTrainingSummary[KMeansSummary] { - - @Since("3.0.0") - lazy val numFeatures: Int = parentModel.clusterCenters.head.size - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeansModel = { - val copied = copyValues(new KMeansModel(uid, parentModel), extra) - copied.setSummary(trainingSummary).setParent(this.parent) - } - - /** @group setParam */ - @Since("2.0.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("2.0.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val predictUDF = udf((vector: Vector) => predict(vector)) - - dataset.withColumn($(predictionCol), - predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)), - outputSchema($(predictionCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(predictionCol).nonEmpty) { - outputSchema = SchemaUtils.updateNumValues(outputSchema, - $(predictionCol), parentModel.k) - } - outputSchema - } - - @Since("3.0.0") - def predict(features: Vector): Int = parentModel.predict(features) - - @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[KMeansModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"KMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + - s"numFeatures=$numFeatures" - } - - /** - * Gets summary of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("2.0.0") - override def summary: KMeansSummary = super.summary -} - -/** Helper class for storing model data */ -private case class ClusterData(clusterIdx: Int, clusterCenter: Vector) - - -/** A writer for KMeans that handles the "internal" (or default) format */ -private class InternalKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: cluster centers - val data: Array[ClusterData] = instance.clusterCenters.zipWithIndex.map { - case (center, idx) => - ClusterData(idx, center) - } - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(data).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for KMeans that handles the "pmml" format */ -private class PMMLKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - instance.parentModel.toPMML(sc, path) - } -} - - -@Since("1.6.0") -object KMeansModel extends MLReadable[KMeansModel] { - - @Since("1.6.0") - override def read: MLReader[KMeansModel] = new KMeansModelReader - - @Since("1.6.0") - override def load(path: String): KMeansModel = super.load(path) - - /** - * We store all cluster centers in a single row and use this class to store model data by - * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. - */ - private case class OldData(clusterCenters: Array[OldVector]) - - private class KMeansModelReader extends MLReader[KMeansModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[KMeansModel].getName - - override def load(path: String): KMeansModel = { - // Import implicits for Dataset Encoder - val sparkSession = super.sparkSession - import sparkSession.implicits._ - - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - - val clusterCenters = if (majorVersion(metadata.sparkVersion) >= 2) { - val data: Dataset[ClusterData] = sparkSession.read.parquet(dataPath).as[ClusterData] - data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) - } else { - // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. - sparkSession.read.parquet(dataPath).as[OldData].head().clusterCenters - } - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) - metadata.getAndSetParams(model) - model - } - } -} - -/** - * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. - * - * @see Bahmani et al., Scalable k-means++. - */ -@Since("1.5.0") -class KMeans @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeans = defaultCopy(extra) - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("kmeans")) - - /** @group setParam */ - @Since("1.5.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitMode(value: String): this.type = set(initMode, value) - - /** @group expertSetParam */ - @Since("2.4.0") - def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitSteps(value: Int): this.type = set(initSteps, value) - - /** @group setParam */ - @Since("1.5.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.5.0") - def setTol(value: Double): this.type = set(tol, value) - - /** @group setParam */ - @Since("1.5.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** - * 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("3.0.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): KMeansModel = instrumented { instr => - transformSchema(dataset.schema, logging = true) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, featuresCol, predictionCol, k, initMode, initSteps, distanceMeasure, - maxIter, seed, tol, weightCol) - - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - val handleWeight = isDefined(weightCol) && $(weightCol).nonEmpty - val w = if (handleWeight) { - col($(weightCol)).cast(DoubleType) - } else { - lit(1.0) - } - - val instances: RDD[(Vector, Double)] = dataset - .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w).rdd.map { - case Row(point: Vector, weight: Double) => (point, weight) - } - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - val useKMeansDAL = Utils.isOAPEnabled() && isPlatformSupported && - $(distanceMeasure) == "euclidean" && !handleWeight - - val model = if (useKMeansDAL) { - trainWithDAL(instances, handlePersistence) - } else { - trainWithML(instances, handlePersistence) - } - - val summary = new KMeansSummary( - model.transform(dataset), - $(predictionCol), - $(featuresCol), - $(k), - model.parentModel.numIter, - model.parentModel.trainingCost) - - model.setSummary(Some(summary)) - instr.logNamedValue("clusterSizes", summary.clusterSizes) - - model - } - - private def trainWithDAL(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - - val sc = instances.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"KMeansDAL fit using $executor_num Executors") - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString($(distanceMeasure)) - - // Use MLlibKMeans to initialize centers - val mllibKMeans = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - - val dataWithNorm = instances.map { - case (point: Vector, weight: Double) => new VectorWithNorm(point) - } - - // Cache for init - dataWithNorm.persist(StorageLevel.MEMORY_AND_DISK) - - val centersWithNorm = if ($(initMode) == "random") { - mllibKMeans.initRandom(dataWithNorm) - } else { - mllibKMeans.initKMeansParallel(dataWithNorm, distanceMeasureInstance) - } - - dataWithNorm.unpersist() - - val centers = centersWithNorm.map(_.vector) - - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - - val strInitMode = $(initMode) - logInfo(f"Initialization with $strInitMode took $initTimeInSeconds%.3f seconds.") - - if (handlePersistence) { - instances.persist(StorageLevel.MEMORY_AND_DISK) - } - - val inputData = instances.map { - case (point: Vector, weight: Double) => point - } - - val kmeansDAL = new KMeansDALImpl(getK, getMaxIter, getTol, - DistanceMeasure.EUCLIDEAN, centers, executor_num, executor_cores) - - val parentModel = kmeansDAL.train(inputData, Option(instr)) - - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - - if (handlePersistence) { - instances.unpersist() - } - - model - } - - private def trainWithML(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - val oldVectorInstances = instances.map { - case (point: Vector, weight: Double) => (OldVectors.fromML(point), weight) - } - val algo = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - val parentModel = algo.runWithWeight(oldVectorInstances, handlePersistence, Some(instr)) - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - - model - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } -} - -@Since("1.6.0") -object KMeans extends DefaultParamsReadable[KMeans] { - - @Since("1.6.0") - override def load(path: String): KMeans = super.load(path) -} - -/** - * Summary of KMeans. - * - * @param predictions `DataFrame` produced by `KMeansModel.transform()`. - * @param predictionCol Name for column of predicted clusters in `predictions`. - * @param featuresCol Name for column of features in `predictions`. - * @param k Number of clusters. - * @param numIter Number of iterations. - * @param trainingCost K-means cost (sum of squared distances to the nearest centroid for all - * points in the training dataset). This is equivalent to sklearn's inertia. - */ -@Since("2.0.0") -class KMeansSummary private[clustering] ( - predictions: DataFrame, - predictionCol: String, - featuresCol: String, - k: Int, - numIter: Int, - @Since("2.4.0") val trainingCost: Double) - extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/feature/PCA.scala deleted file mode 100644 index 8ca5a9b5d..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/feature/PCA.scala +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.feature - -import org.apache.hadoop.fs.Path - -import org.apache.spark.annotation.Since -import org.apache.spark.ml._ -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util._ -import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} -import org.apache.spark.sql._ -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.VersionUtils.majorVersion - -/** - * Params for [[PCA]] and [[PCAModel]]. - */ -private[feature] trait PCAParams extends Params with HasInputCol with HasOutputCol { - - /** - * The number of principal components. - * @group param - */ - final val k: IntParam = new IntParam(this, "k", "the number of principal components (> 0)", - ParamValidators.gt(0)) - - /** @group getParam */ - def getK: Int = $(k) - - /** Validates and transforms the input schema. */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - SchemaUtils.updateAttributeGroupSize(schema, $(outputCol), $(k)) - } -} - -/** - * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` - * principal components. - */ -@Since("1.5.0") -class PCA @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("pca")) - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** - * Computes a [[PCAModel]] that contains the principal components of the input vectors. - */ - @Since("2.0.0") - override def fit(dataset: Dataset[_]): PCAModel = { - transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd - val inputVectors = input.map { - case Row(v: Vector) => v - } - - val numFeatures = inputVectors.first().size - require($(k) <= numFeatures, - s"source vector size $numFeatures must be no less than k=$k") - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - - // Call oneDAL Correlation PCA implementation when numFeatures < 65535 and fall back otherwise - val parentModel = if (numFeatures < 65535 && Utils.isOAPEnabled() && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val pca = new PCADALImpl(k = $(k), executor_num, executor_cores) - val pcaModel = pca.train(inputVectors) - pcaModel - } else { - val inputOldVectors = inputVectors.map { - case v: Vector => OldVectors.fromML(v) - } - val pca = new feature.PCA(k = $(k)) - val pcaModel = pca.fit(inputOldVectors) - pcaModel - } - copyValues(new PCAModel(uid, parentModel.pc.asML, parentModel.explainedVariance.asML) - .setParent(this)) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCA = defaultCopy(extra) -} - -@Since("1.6.0") -object PCA extends DefaultParamsReadable[PCA] { - - @Since("1.6.0") - override def load(path: String): PCA = super.load(path) -} - -/** - * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. - * - * @param pc A principal components Matrix. Each column is one principal component. - * @param explainedVariance A vector of proportions of variance explained by - * each principal component. - */ -@Since("1.5.0") -class PCAModel private[ml] ( - @Since("1.5.0") override val uid: String, - @Since("2.0.0") val pc: DenseMatrix, - @Since("2.0.0") val explainedVariance: DenseVector) - extends Model[PCAModel] with PCAParams with MLWritable { - - import PCAModel._ - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** - * Transform a vector by computed Principal Components. - * - * @note Vectors to be transformed must be the same length as the source vectors given - * to `PCA.fit()`. - */ - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val transposed = pc.transpose - val transformer = udf { vector: Vector => transposed.multiply(vector) } - dataset.withColumn($(outputCol), transformer(col($(inputCol))), - outputSchema($(outputCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(outputCol).nonEmpty) { - outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, - $(outputCol), $(k)) - } - outputSchema - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCAModel = { - val copied = new PCAModel(uid, pc, explainedVariance) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new PCAModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"PCAModel: uid=$uid, k=${$(k)}" - } -} - -@Since("1.6.0") -object PCAModel extends MLReadable[PCAModel] { - - private[PCAModel] class PCAModelWriter(instance: PCAModel) extends MLWriter { - - private case class Data(pc: DenseMatrix, explainedVariance: DenseVector) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.pc, instance.explainedVariance) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class PCAModelReader extends MLReader[PCAModel] { - - private val className = classOf[PCAModel].getName - - /** - * Loads a [[PCAModel]] from data located at the input path. Note that the model includes an - * `explainedVariance` member that is not recorded by Spark 1.6 and earlier. A model - * can be loaded from such older data but will have an empty vector for - * `explainedVariance`. - * - * @param path path to serialized model data - * @return a [[PCAModel]] - */ - override def load(path: String): PCAModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val model = if (majorVersion(metadata.sparkVersion) >= 2) { - val Row(pc: DenseMatrix, explainedVariance: DenseVector) = - sparkSession.read.parquet(dataPath) - .select("pc", "explainedVariance") - .head() - new PCAModel(metadata.uid, pc, explainedVariance) - } else { - // pc field is the old matrix format in Spark <= 1.6 - // explainedVariance field is not present in Spark <= 1.6 - val Row(pc: OldDenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() - new PCAModel(metadata.uid, pc.asML, new DenseVector(Array.emptyDoubleArray)) - } - metadata.getAndSetParams(model) - model - } - } - - @Since("1.6.0") - override def read: MLReader[PCAModel] = new PCAModelReader - - @Since("1.6.0") - override def load(path: String): PCAModel = super.load(path) -} diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala deleted file mode 100644 index 62e5e3283..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ /dev/null @@ -1,1059 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.regression - -import scala.collection.mutable - -import breeze.linalg.{DenseVector => BDV} -import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, LBFGSB => BreezeLBFGSB, OWLQN => BreezeOWLQN} -import breeze.stats.distributions.StudentsT -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkException -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PipelineStage, PredictorParams} -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{Vector, Vectors} -import org.apache.spark.ml.linalg.BLAS._ -import org.apache.spark.ml.optim.WeightedLeastSquares -import org.apache.spark.ml.optim.aggregator.{HuberAggregator, LeastSquaresAggregator} -import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} -import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.stat._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.regression.{LinearRegressionModel => OldLinearRegressionModel} -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, DoubleType, StructType} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorMinorVersion - -/** - * Params for linear regression. - */ -private[regression] trait LinearRegressionParams extends PredictorParams - with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol - with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver - with HasAggregationDepth with HasLoss { - - import LinearRegression._ - - /** - * The solver algorithm for optimization. - * Supported options: "l-bfgs", "normal" and "auto". - * Default: "auto" - * - * @group param - */ - @Since("1.6.0") - final override val solver: Param[String] = new Param[String](this, "solver", - "The solver algorithm for optimization. Supported options: " + - s"${supportedSolvers.mkString(", ")}. (Default auto)", - ParamValidators.inArray[String](supportedSolvers)) - - /** - * The loss function to be optimized. - * Supported options: "squaredError" and "huber". - * Default: "squaredError" - * - * @group param - */ - @Since("2.3.0") - final override val loss: Param[String] = new Param[String](this, "loss", "The loss function to" + - s" be optimized. Supported options: ${supportedLosses.mkString(", ")}. (Default squaredError)", - ParamValidators.inArray[String](supportedLosses)) - - /** - * The shape parameter to control the amount of robustness. Must be > 1.0. - * At larger values of epsilon, the huber criterion becomes more similar to least squares - * regression; for small values of epsilon, the criterion is more similar to L1 regression. - * Default is 1.35 to get as much robustness as possible while retaining - * 95% statistical efficiency for normally distributed data. It matches sklearn - * HuberRegressor and is "M" from - * A robust hybrid of lasso and ridge regression. - * Only valid when "loss" is "huber". - * - * @group expertParam - */ - @Since("2.3.0") - final val epsilon = new DoubleParam(this, "epsilon", "The shape parameter to control the " + - "amount of robustness. Must be > 1.0.", ParamValidators.gt(1.0)) - - /** @group getExpertParam */ - @Since("2.3.0") - def getEpsilon: Double = $(epsilon) - - setDefault(regParam -> 0.0, fitIntercept -> true, standardization -> true, - elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, solver -> Auto, - aggregationDepth -> 2, loss -> SquaredError, epsilon -> 1.35) - - override protected def validateAndTransformSchema( - schema: StructType, - fitting: Boolean, - featuresDataType: DataType): StructType = { - if (fitting) { - if ($(loss) == Huber) { - require($(solver)!= Normal, "LinearRegression with huber loss doesn't support " + - "normal solver, please change solver to auto or l-bfgs.") - require($(elasticNetParam) == 0.0, "LinearRegression with huber loss only supports " + - s"L2 regularization, but got elasticNetParam = $getElasticNetParam.") - } - } - super.validateAndTransformSchema(schema, fitting, featuresDataType) - } -} - -/** - * Linear regression. - * - * The learning objective is to minimize the specified loss function, with regularization. - * This supports two kinds of loss: - * - squaredError (a.k.a squared loss) - * - huber (a hybrid of squared error for relatively small errors and absolute error for - * relatively large ones, and we estimate the scale parameter from training data) - * - * This supports multiple types of regularization: - * - none (a.k.a. ordinary least squares) - * - L2 (ridge regression) - * - L1 (Lasso) - * - L2 + L1 (elastic net) - * - * The squared error objective function is: - * - *
- * $$ - * \begin{align} - * \min_{w}\frac{1}{2n}{\sum_{i=1}^n(X_{i}w - y_{i})^{2} + - * \lambda\left[\frac{1-\alpha}{2}{||w||_{2}}^{2} + \alpha{||w||_{1}}\right]} - * \end{align} - * $$ - *
- * - * The huber objective function is: - * - *
- * $$ - * \begin{align} - * \min_{w, \sigma}\frac{1}{2n}{\sum_{i=1}^n\left(\sigma + - * H_m\left(\frac{X_{i}w - y_{i}}{\sigma}\right)\sigma\right) + \frac{1}{2}\lambda {||w||_2}^2} - * \end{align} - * $$ - *
- * - * where - * - *
- * $$ - * \begin{align} - * H_m(z) = \begin{cases} - * z^2, & \text {if } |z| < \epsilon, \\ - * 2\epsilon|z| - \epsilon^2, & \text{otherwise} - * \end{cases} - * \end{align} - * $$ - *
- * - * Note: Fitting with huber loss only supports none and L2 regularization. - */ -@Since("1.3.0") -class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) - extends Regressor[Vector, LinearRegression, LinearRegressionModel] - with LinearRegressionParams with DefaultParamsWritable with Logging { - - import LinearRegression._ - - @Since("1.4.0") - def this() = this(Identifiable.randomUID("linReg")) - - /** - * Set the regularization parameter. - * Default is 0.0. - * - * @group setParam - */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - - /** - * Set if we should fit the intercept. - * Default is true. - * - * @group setParam - */ - @Since("1.5.0") - def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - - /** - * Whether to standardize the training features before fitting the model. - * The coefficients of models will be always returned on the original scale, - * so it will be transparent for users. - * Default is true. - * - * @note With/without standardization, the models should be always converged - * to the same solution when no regularization is applied. In R's GLMNET package, - * the default behavior is true as well. - * - * @group setParam - */ - @Since("1.5.0") - def setStandardization(value: Boolean): this.type = set(standardization, value) - - /** - * Set the ElasticNet mixing parameter. - * For alpha = 0, the penalty is an L2 penalty. - * For alpha = 1, it is an L1 penalty. - * For alpha in (0,1), the penalty is a combination of L1 and L2. - * Default is 0.0 which is an L2 penalty. - * - * Note: Fitting with huber loss only supports None and L2 regularization, - * so throws exception if this param is non-zero value. - * - * @group setParam - */ - @Since("1.4.0") - def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - - /** - * Set the maximum number of iterations. - * Default is 100. - * - * @group setParam - */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** - * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. - * Default is 1E-6. - * - * @group setParam - */ - @Since("1.4.0") - def setTol(value: Double): this.type = set(tol, value) - - /** - * Whether to over-/under-sample training instances according to the given weights in weightCol. - * If not set or empty, all instances are treated equally (weight 1.0). - * Default is not set, so all instances have weight one. - * - * @group setParam - */ - @Since("1.6.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - /** - * Set the solver algorithm used for optimization. - * In case of linear regression, this can be "l-bfgs", "normal" and "auto". - * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton - * optimization method. - * - "normal" denotes using Normal Equation as an analytical solution to the linear regression - * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. - * - "auto" (default) means that the solver algorithm is selected automatically. - * The Normal Equations solver will be used when possible, but this will automatically fall - * back to iterative optimization methods when needed. - * - * Note: Fitting with huber loss doesn't support normal solver, - * so throws exception if this param was set with "normal". - * @group setParam - */ - @Since("1.6.0") - def setSolver(value: String): this.type = set(solver, value) - - /** - * Suggested depth for treeAggregate (greater than or equal to 2). - * If the dimensions of features or the number of partitions are large, - * this param could be adjusted to a larger size. - * Default is 2. - * - * @group expertSetParam - */ - @Since("2.1.0") - def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - - /** - * Sets the value of param [[loss]]. - * Default is "squaredError". - * - * @group setParam - */ - @Since("2.3.0") - def setLoss(value: String): this.type = set(loss, value) - - /** - * Sets the value of param [[epsilon]]. - * Default is 1.35. - * - * @group setExpertParam - */ - @Since("2.3.0") - def setEpsilon(value: Double): this.type = set(epsilon, value) - - override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => - // Extract the number of features before deciding optimization solver. - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) - - val instances = extractInstances(dataset) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, solver, tol, - elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth, loss, - epsilon) - instr.logNumFeatures(numFeatures) - - if ($(loss) == SquaredError && (($(solver) == Auto && - numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == Normal)) { - // oneDAL only support simple linear regression and ridge regression - val paramSupported = ($(regParam) == 0) || ($(regParam) != 0 && $(elasticNetParam) == 0) - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - if (paramSupported && Utils.isOAPEnabled && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - logInfo(s"LinearRegressionDAL fit using $executor_num Executors") - - val optimizer = new LinearRegressionDALImpl($(fitIntercept), $(regParam), - elasticNetParam = $(elasticNetParam), $(standardization), true, - executor_num, executor_cores) - - // Return same model as WeightedLeastSquaresModel - val model = optimizer.train(dataset, Some(instr)) - - val lrModel = copyValues( - new LinearRegressionModel(uid, model.coefficients, model.intercept)) - - val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - summaryModel, - model.diagInvAtWA.toArray, - model.objectiveHistory) - - return lrModel.setSummary(Some(trainingSummary)) - } else { - // For low dimensional data, WeightedLeastSquares is more efficient since the - // training algorithm only requires one pass through the data. (SPARK-10668) - - val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), - elasticNetParam = $(elasticNetParam), $(standardization), true, - solverType = WeightedLeastSquares.Auto, maxIter = $(maxIter), tol = $(tol)) - val model = optimizer.fit(instances, instr = OptionalInstrumentation.create(instr)) - // When it is trained by WeightedLeastSquares, training summary does not - // attach returned model. - val lrModel = copyValues(new LinearRegressionModel(uid, model.coefficients, model.intercept)) - val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - summaryModel, - model.diagInvAtWA.toArray, - model.objectiveHistory) - - return lrModel.setSummary(Some(trainingSummary)) - } - } - - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - - val (featuresSummarizer, ySummarizer) = instances.treeAggregate( - (Summarizer.createSummarizerBuffer("mean", "std"), - Summarizer.createSummarizerBuffer("mean", "std", "count")))( - seqOp = (c: (SummarizerBuffer, SummarizerBuffer), instance: Instance) => - (c._1.add(instance.features, instance.weight), - c._2.add(Vectors.dense(instance.label), instance.weight)), - combOp = (c1: (SummarizerBuffer, SummarizerBuffer), - c2: (SummarizerBuffer, SummarizerBuffer)) => - (c1._1.merge(c2._1), c1._2.merge(c2._2)), - depth = $(aggregationDepth) - ) - - val yMean = ySummarizer.mean(0) - val rawYStd = ySummarizer.std(0) - - instr.logNumExamples(ySummarizer.count) - instr.logNamedValue(Instrumentation.loggerTags.meanOfLabels, yMean) - instr.logNamedValue(Instrumentation.loggerTags.varianceOfLabels, rawYStd) - instr.logSumOfWeights(featuresSummarizer.weightSum) - - if (rawYStd == 0.0) { - if ($(fitIntercept) || yMean == 0.0) { - // If the rawYStd==0 and fitIntercept==true, then the intercept is yMean with - // zero coefficient; as a result, training is not needed. - // Also, if yMean==0 and rawYStd==0, all the coefficients are zero regardless of - // the fitIntercept. - if (yMean == 0.0) { - instr.logWarning(s"Mean and standard deviation of the label are zero, so the " + - s"coefficients and the intercept will all be zero; as a result, training is not " + - s"needed.") - } else { - instr.logWarning(s"The standard deviation of the label is zero, so the coefficients " + - s"will be zeros and the intercept will be the mean of the label; as a result, " + - s"training is not needed.") - } - if (handlePersistence) instances.unpersist() - val coefficients = Vectors.sparse(numFeatures, Seq.empty) - val intercept = yMean - - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - model, - Array(0D), - Array(0D)) - - return model.setSummary(Some(trainingSummary)) - } else { - require($(regParam) == 0.0, "The standard deviation of the label is zero. " + - "Model cannot be regularized.") - instr.logWarning(s"The standard deviation of the label is zero. " + - "Consider setting fitIntercept=true.") - } - } - - // if y is constant (rawYStd is zero), then y cannot be scaled. In this case - // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. - val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) - val featuresMean = featuresSummarizer.mean.toArray - val featuresStd = featuresSummarizer.std.toArray - val bcFeaturesMean = instances.context.broadcast(featuresMean) - val bcFeaturesStd = instances.context.broadcast(featuresStd) - - if (!$(fitIntercept) && (0 until numFeatures).exists { i => - featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { - instr.logWarning("Fitting LinearRegressionModel without intercept on dataset with " + - "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + - "columns. This behavior is the same as R glmnet but different from LIBSVM.") - } - - // Since we implicitly do the feature scaling when we compute the cost function - // to improve the convergence, the effective regParam will be changed. - val effectiveRegParam = $(loss) match { - case SquaredError => $(regParam) / yStd - case Huber => $(regParam) - } - val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam - val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam - - val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 - val regularization = if (effectiveL2RegParam != 0.0) { - val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures - Some(new L2Regularization(effectiveL2RegParam, shouldApply, - if ($(standardization)) None else Some(getFeaturesStd))) - } else { - None - } - - val costFun = $(loss) match { - case SquaredError => - val getAggregatorFunc = new LeastSquaresAggregator(yStd, yMean, $(fitIntercept), - bcFeaturesStd, bcFeaturesMean)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) - case Huber => - val getAggregatorFunc = new HuberAggregator($(fitIntercept), $(epsilon), bcFeaturesStd)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) - } - - val optimizer = $(loss) match { - case SquaredError => - if ($(elasticNetParam) == 0.0 || effectiveRegParam == 0.0) { - new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) - } else { - val standardizationParam = $(standardization) - def effectiveL1RegFun = (index: Int) => { - if (standardizationParam) { - effectiveL1RegParam - } else { - // If `standardization` is false, we still standardize the data - // to improve the rate of convergence; as a result, we have to - // perform this reverse standardization by penalizing each component - // differently to get effectively the same objective function when - // the training dataset is not standardized. - if (featuresStd(index) != 0.0) effectiveL1RegParam / featuresStd(index) else 0.0 - } - } - new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, effectiveL1RegFun, $(tol)) - } - case Huber => - val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 - val lowerBounds = BDV[Double](Array.fill(dim)(Double.MinValue)) - // Optimize huber loss in space "\sigma > 0" - lowerBounds(dim - 1) = Double.MinPositiveValue - val upperBounds = BDV[Double](Array.fill(dim)(Double.MaxValue)) - new BreezeLBFGSB(lowerBounds, upperBounds, $(maxIter), 10, $(tol)) - } - - val initialValues = $(loss) match { - case SquaredError => - Vectors.zeros(numFeatures) - case Huber => - val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 - Vectors.dense(Array.fill(dim)(1.0)) - } - - val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialValues.asBreeze.toDenseVector) - - val (coefficients, intercept, scale, objectiveHistory) = { - /* - Note that in Linear Regression, the objective history (loss + regularization) returned - from optimizer is computed in the scaled space given by the following formula. -
- $$ - L &= 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 - + regTerms \\ - $$ -
- */ - val arrayBuilder = mutable.ArrayBuilder.make[Double] - var state: optimizer.State = null - while (states.hasNext) { - state = states.next() - arrayBuilder += state.adjustedValue - } - if (state == null) { - val msg = s"${optimizer.getClass.getName} failed." - instr.logError(msg) - throw new SparkException(msg) - } - - bcFeaturesMean.destroy() - bcFeaturesStd.destroy() - - val parameters = state.x.toArray.clone() - - /* - The coefficients are trained in the scaled space; we're converting them back to - the original space. - */ - val rawCoefficients: Array[Double] = $(loss) match { - case SquaredError => parameters - case Huber => parameters.slice(0, numFeatures) - } - - var i = 0 - val len = rawCoefficients.length - val multiplier = $(loss) match { - case SquaredError => yStd - case Huber => 1.0 - } - while (i < len) { - rawCoefficients(i) *= { if (featuresStd(i) != 0.0) multiplier / featuresStd(i) else 0.0 } - i += 1 - } - - val interceptValue: Double = if ($(fitIntercept)) { - $(loss) match { - case SquaredError => - /* - The intercept of squared error in R's GLMNET is computed using closed form - after the coefficients are converged. See the following discussion for detail. - http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet - */ - yMean - dot(Vectors.dense(rawCoefficients), Vectors.dense(featuresMean)) - case Huber => parameters(numFeatures) - } - } else { - 0.0 - } - - val scaleValue: Double = $(loss) match { - case SquaredError => 1.0 - case Huber => parameters.last - } - - (Vectors.dense(rawCoefficients).compressed, interceptValue, scaleValue, arrayBuilder.result()) - } - - if (handlePersistence) instances.unpersist() - - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept, scale)) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - model, - Array(0D), - objectiveHistory) - - model.setSummary(Some(trainingSummary)) - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra) -} - -@Since("1.6.0") -object LinearRegression extends DefaultParamsReadable[LinearRegression] { - - @Since("1.6.0") - override def load(path: String): LinearRegression = super.load(path) - - /** - * When using `LinearRegression.solver` == "normal", the solver must limit the number of - * features to at most this number. The entire covariance matrix X^T^X will be collected - * to the driver. This limit helps prevent memory overflow errors. - */ - @Since("2.1.0") - val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES - - /** String name for "auto". */ - private[regression] val Auto = "auto" - - /** String name for "normal". */ - private[regression] val Normal = "normal" - - /** String name for "l-bfgs". */ - private[regression] val LBFGS = "l-bfgs" - - /** Set of solvers that LinearRegression supports. */ - private[regression] val supportedSolvers = Array(Auto, Normal, LBFGS) - - /** String name for "squaredError". */ - private[regression] val SquaredError = "squaredError" - - /** String name for "huber". */ - private[regression] val Huber = "huber" - - /** Set of loss function names that LinearRegression supports. */ - private[regression] val supportedLosses = Array(SquaredError, Huber) -} - -/** - * Model produced by [[LinearRegression]]. - */ -@Since("1.3.0") -class LinearRegressionModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("2.0.0") val coefficients: Vector, - @Since("1.3.0") val intercept: Double, - @Since("2.3.0") val scale: Double) - extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams with GeneralMLWritable - with HasTrainingSummary[LinearRegressionTrainingSummary] { - - private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = - this(uid, coefficients, intercept, 1.0) - - override val numFeatures: Int = coefficients.size - - /** - * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("1.5.0") - override def summary: LinearRegressionTrainingSummary = super.summary - - /** - * Evaluates the model on a test dataset. - * - * @param dataset Test dataset to evaluate model on. - */ - @Since("2.0.0") - def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() - new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, - $(labelCol), $(featuresCol), summaryModel, Array(0D)) - } - - /** - * If the prediction column is set returns the current model and prediction column, - * otherwise generates a new column and sets it as the prediction column on a new copy - * of the current model. - */ - private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { - $(predictionCol) match { - case "" => - val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString - (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) - case p => (this, p) - } - } - - - override def predict(features: Vector): Double = { - dot(features, coefficients) + intercept - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegressionModel = { - val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) - newModel.setSummary(trainingSummary).setParent(parent) - } - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - * This also does not save the [[parent]] currently. - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" - } -} - -/** A writer for LinearRegression that handles the "internal" (or default) format */ -private class InternalLinearRegressionModelWriter - extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" - - private case class Data(intercept: Double, coefficients: Vector, scale: Double) - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[LinearRegressionModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: intercept, coefficients, scale - val data = Data(instance.intercept, instance.coefficients, instance.scale) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for LinearRegression that handles the "pmml" format */ -private class PMMLLinearRegressionModelWriter - extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - - override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" - - private case class Data(intercept: Double, coefficients: Vector) - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val sc = sparkSession.sparkContext - // Construct the MLLib model which knows how to write to PMML. - val instance = stage.asInstanceOf[LinearRegressionModel] - val oldModel = new OldLinearRegressionModel(instance.coefficients, instance.intercept) - // Save PMML - oldModel.toPMML(sc, path) - } -} - -@Since("1.6.0") -object LinearRegressionModel extends MLReadable[LinearRegressionModel] { - - @Since("1.6.0") - override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader - - @Since("1.6.0") - override def load(path: String): LinearRegressionModel = super.load(path) - - private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[LinearRegressionModel].getName - - override def load(path: String): LinearRegressionModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.format("parquet").load(dataPath) - val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) - val model = if (majorVersion < 2 || (majorVersion == 2 && minorVersion <= 2)) { - // Spark 2.2 and before - val Row(intercept: Double, coefficients: Vector) = - MLUtils.convertVectorColumnsToML(data, "coefficients") - .select("intercept", "coefficients") - .head() - new LinearRegressionModel(metadata.uid, coefficients, intercept) - } else { - // Spark 2.3 and later - val Row(intercept: Double, coefficients: Vector, scale: Double) = - data.select("intercept", "coefficients", "scale").head() - new LinearRegressionModel(metadata.uid, coefficients, intercept, scale) - } - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Linear regression training results. Currently, the training summary ignores the - * training weights except for the objective trace. - * - * @param predictions predictions output by the model's `transform` method. - * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. - */ -@Since("1.5.0") -class LinearRegressionTrainingSummary private[regression] ( - predictions: DataFrame, - predictionCol: String, - labelCol: String, - featuresCol: String, - model: LinearRegressionModel, - diagInvAtWA: Array[Double], - val objectiveHistory: Array[Double]) - extends LinearRegressionSummary( - predictions, - predictionCol, - labelCol, - featuresCol, - model, - diagInvAtWA) { - - /** - * Number of training iterations until termination - * - * This value is only available when using the "l-bfgs" solver. - * - * @see `LinearRegression.solver` - */ - @Since("1.5.0") - val totalIterations = objectiveHistory.length - -} - -/** - * Linear regression results evaluated on a dataset. - * - * @param predictions predictions output by the model's `transform` method. - * @param predictionCol Field in "predictions" which gives the predicted value of the label at - * each instance. - * @param labelCol Field in "predictions" which gives the true label of each instance. - * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. - */ -@Since("1.5.0") -class LinearRegressionSummary private[regression] ( - @transient val predictions: DataFrame, - val predictionCol: String, - val labelCol: String, - val featuresCol: String, - private val privateModel: LinearRegressionModel, - private val diagInvAtWA: Array[Double]) extends Serializable { - - @transient private val metrics = new RegressionMetrics( - predictions - .select(col(predictionCol), col(labelCol).cast(DoubleType)) - .rdd - .map { case Row(pred: Double, label: Double) => (pred, label) }, - !privateModel.getFitIntercept) - - /** - * Returns the explained variance regression score. - * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) - * Reference: - * Wikipedia explain variation - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val explainedVariance: Double = metrics.explainedVariance - - /** - * Returns the mean absolute error, which is a risk function corresponding to the - * expected value of the absolute error loss or l1-norm loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val meanAbsoluteError: Double = metrics.meanAbsoluteError - - /** - * Returns the mean squared error, which is a risk function corresponding to the - * expected value of the squared error loss or quadratic loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val meanSquaredError: Double = metrics.meanSquaredError - - /** - * Returns the root mean squared error, which is defined as the square root of - * the mean squared error. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val rootMeanSquaredError: Double = metrics.rootMeanSquaredError - - /** - * Returns R^2^, the coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val r2: Double = metrics.r2 - - /** - * Returns Adjusted R^2^, the adjusted coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("2.3.0") - val r2adj: Double = { - val interceptDOF = if (privateModel.getFitIntercept) 1 else 0 - 1 - (1 - r2) * (numInstances - interceptDOF) / - (numInstances - privateModel.coefficients.size - interceptDOF) - } - - /** Residuals (label - predicted value) */ - @Since("1.5.0") - @transient lazy val residuals: DataFrame = { - val t = udf { (pred: Double, label: Double) => label - pred } - predictions.select(t(col(predictionCol), col(labelCol)).as("residuals")) - } - - /** Number of instances in DataFrame predictions */ - lazy val numInstances: Long = predictions.count() - - /** Degrees of freedom */ - @Since("2.2.0") - val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { - numInstances - privateModel.coefficients.size - 1 - } else { - numInstances - privateModel.coefficients.size - } - - /** - * The weighted residuals, the usual residuals rescaled by - * the square root of the instance weights. - */ - lazy val devianceResiduals: Array[Double] = { - val weighted = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - lit(1.0) - } else { - sqrt(col(privateModel.getWeightCol)) - } - val dr = predictions - .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) - .multiply(weighted).as("weightedResiduals")) - .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) - .first() - Array(dr.getDouble(0), dr.getDouble(1)) - } - - /** - * Standard error of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val coefficientStandardErrors: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No Std. Error of coefficients available for this LinearRegressionModel") - } else { - val rss = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - meanSquaredError * numInstances - } else { - val t = udf { (pred: Double, label: Double, weight: Double) => - math.pow(label - pred, 2.0) * weight } - predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), - col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) - } - val sigma2 = rss / degreesOfFreedom - diagInvAtWA.map(_ * sigma2).map(math.sqrt) - } - } - - /** - * T-statistic of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val tValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No t-statistic available for this LinearRegressionModel") - } else { - val estimate = if (privateModel.getFitIntercept) { - Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) - } else { - privateModel.coefficients.toArray - } - estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } - } - } - - /** - * Two-sided p-value of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val pValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No p-value available for this LinearRegressionModel") - } else { - tValues.map { x => 2.0 * (1.0 - StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) } - } - } - -} - diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/stat/Correlation.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/stat/Correlation.scala deleted file mode 100644 index 13e1f4993..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/ml/stat/Correlation.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.stat - -import org.apache.spark.annotation.{Experimental, Since} -import org.apache.spark.ml.linalg.{SQLDataTypes, Vector} -import org.apache.spark.ml.util.Utils -import org.apache.spark.mllib.linalg.{Vectors => OldVectors} -import org.apache.spark.mllib.stat.{Statistics => OldStatistics} -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, Row} -import org.apache.spark.storage.StorageLevel - -import scala.collection.JavaConverters._ - -/** - * API for correlation functions in MLlib, compatible with DataFrames and Datasets. - * - * The functions in this package generalize the functions in [[org.apache.spark.sql.Dataset#stat]] - * to spark.ml's Vector types. - */ -@Since("2.2.0") -@Experimental -object Correlation { - - /** - * :: Experimental :: - * Compute the correlation matrix for the input Dataset of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param dataset A dataset or a dataframe - * @param column The name of the column of vectors for which the correlation coefficient needs - * to be computed. This must be a column of the dataset, and it must contain - * Vector objects. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return A dataframe that contains the correlation matrix of the column of vectors. This - * dataframe contains a single row and a single column of name - * '$METHODNAME($COLUMN)'. - * @throws IllegalArgumentException if the column is not a valid column in the dataset, or if - * the content of this column is not of type Vector. - * - * Here is how to access the correlation coefficient: - * {{{ - * val data: Dataset[Vector] = ... - * val Row(coeff: Matrix) = Correlation.corr(data, "value").head - * // coeff now contains the Pearson correlation matrix. - * }}} - * - * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input Dataset before calling corr with `method = "spearman"` - * to avoid recomputing the common lineage. - */ - @Since("2.2.0") - def corr(dataset: Dataset[_], column: String, method: String): DataFrame = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - if (Utils.isOAPEnabled() && isPlatformSupported && method == "pearson") { - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - if (handlePersistence) { - dataset.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = dataset.select(column).rdd.map { - case Row(v: Vector) => v - } - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val matrix = new CorrelationDALImpl(executor_num, executor_cores) - .computeCorrelationMatrix(rdd) - val name = s"$method($column)" - val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) - val dataframe = dataset.sparkSession.createDataFrame(Seq(Row(matrix)).asJava, schema) - if (handlePersistence) { - dataset.unpersist() - } - dataframe - } else { - val rdd = dataset.select(column).rdd.map { - case Row(v: Vector) => OldVectors.fromML(v) - } - val oldM = OldStatistics.corr(rdd, method) - val name = s"$method($column)" - val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) - dataset.sparkSession.createDataFrame(Seq(Row(oldM.asML)).asJava, schema) - } - } - - /** - * Compute the Pearson correlation matrix for the input Dataset of Vectors. - */ - @Since("2.2.0") - def corr(dataset: Dataset[_], column: String): DataFrame = { - corr(dataset, column, "pearson") - } -} diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/clustering/KMeans.scala deleted file mode 100644 index a0f050c65..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ /dev/null @@ -1,520 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.clustering - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.annotation.Since -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.Logging -import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS.axpy -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils -import org.apache.spark.util.random.XORShiftRandom - -/** - * K-means clustering with a k-means++ like initialization mode - * (the k-means|| algorithm by Bahmani et al). - * - * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given - * to it should be cached by the user. - */ -@Since("0.8.0") -class KMeans private ( - private var k: Int, - private var maxIterations: Int, - private var initializationMode: String, - private var initializationSteps: Int, - private var epsilon: Double, - private var seed: Long, - private var distanceMeasure: String) extends Serializable with Logging { - - @Since("0.8.0") - private def this(k: Int, maxIterations: Int, initializationMode: String, initializationSteps: Int, - epsilon: Double, seed: Long) = - this(k, maxIterations, initializationMode, initializationSteps, - epsilon, seed, DistanceMeasure.EUCLIDEAN) - - /** - * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, - * initializationMode: "k-means||", initializationSteps: 2, epsilon: 1e-4, seed: random, - * distanceMeasure: "euclidean"}. - */ - @Since("0.8.0") - def this() = this(2, 20, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong(), - DistanceMeasure.EUCLIDEAN) - - /** - * Number of clusters to create (k). - * - * @note It is possible for fewer than k clusters to - * be returned, for example, if there are fewer than k distinct points to cluster. - */ - @Since("1.4.0") - def getK: Int = k - - /** - * Set the number of clusters to create (k). - * - * @note It is possible for fewer than k clusters to - * be returned, for example, if there are fewer than k distinct points to cluster. Default: 2. - */ - @Since("0.8.0") - def setK(k: Int): this.type = { - require(k > 0, - s"Number of clusters must be positive but got ${k}") - this.k = k - this - } - - /** - * Maximum number of iterations allowed. - */ - @Since("1.4.0") - def getMaxIterations: Int = maxIterations - - /** - * Set maximum number of iterations allowed. Default: 20. - */ - @Since("0.8.0") - def setMaxIterations(maxIterations: Int): this.type = { - require(maxIterations >= 0, - s"Maximum of iterations must be nonnegative but got ${maxIterations}") - this.maxIterations = maxIterations - this - } - - /** - * The initialization algorithm. This can be either "random" or "k-means||". - */ - @Since("1.4.0") - def getInitializationMode: String = initializationMode - - /** - * Set the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - */ - @Since("0.8.0") - def setInitializationMode(initializationMode: String): this.type = { - KMeans.validateInitMode(initializationMode) - this.initializationMode = initializationMode - this - } - - /** - * Number of steps for the k-means|| initialization mode - */ - @Since("1.4.0") - def getInitializationSteps: Int = initializationSteps - - /** - * Set the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Default: 2. - */ - @Since("0.8.0") - def setInitializationSteps(initializationSteps: Int): this.type = { - require(initializationSteps > 0, - s"Number of initialization steps must be positive but got ${initializationSteps}") - this.initializationSteps = initializationSteps - this - } - - /** - * The distance threshold within which we've consider centers to have converged. - */ - @Since("1.4.0") - def getEpsilon: Double = epsilon - - /** - * Set the distance threshold within which we've consider centers to have converged. - * If all centers move less than this Euclidean distance, we stop iterating one run. - */ - @Since("0.8.0") - def setEpsilon(epsilon: Double): this.type = { - require(epsilon >= 0, - s"Distance threshold must be nonnegative but got ${epsilon}") - this.epsilon = epsilon - this - } - - /** - * The random seed for cluster initialization. - */ - @Since("1.4.0") - def getSeed: Long = seed - - /** - * Set the random seed for cluster initialization. - */ - @Since("1.4.0") - def setSeed(seed: Long): this.type = { - this.seed = seed - this - } - - /** - * The distance suite used by the algorithm. - */ - @Since("2.4.0") - def getDistanceMeasure: String = distanceMeasure - - /** - * Set the distance suite used by the algorithm. - */ - @Since("2.4.0") - def setDistanceMeasure(distanceMeasure: String): this.type = { - DistanceMeasure.validateDistanceMeasure(distanceMeasure) - this.distanceMeasure = distanceMeasure - this - } - - // Initial cluster centers can be provided as a KMeansModel object rather than using the - // random or k-means|| initializationMode - private var initialModel: Option[KMeansModel] = None - - /** - * Set the initial starting point, bypassing the random initialization or k-means|| - * The condition model.k == this.k must be met, failure results - * in an IllegalArgumentException. - */ - @Since("1.4.0") - def setInitialModel(model: KMeansModel): this.type = { - require(model.k == k, "mismatched cluster count") - initialModel = Some(model) - this - } - - /** - * Train a K-means model on the given set of points; `data` should be cached for high - * performance, because this is an iterative algorithm. - */ - @Since("0.8.0") - def run(data: RDD[Vector]): KMeansModel = { - val instances = data.map(point => (point, 1.0)) - val handlePersistence = data.getStorageLevel == StorageLevel.NONE - runWithWeight(instances, handlePersistence, None) - } - - private[spark] def runWithWeight( - instances: RDD[(Vector, Double)], - handlePersistence: Boolean, - instr: Option[Instrumentation]): KMeansModel = { - val norms = instances.map { case (v, _) => Vectors.norm(v, 2.0) } - val vectors = instances.zip(norms) - .map { case ((v, w), norm) => new VectorWithNorm(v, norm, w) } - - if (handlePersistence) { - vectors.persist(StorageLevel.MEMORY_AND_DISK) - } else { - // Compute squared norms and cache them. - norms.persist(StorageLevel.MEMORY_AND_DISK) - } - val model = runAlgorithmWithWeight(vectors, instr) - if (handlePersistence) { vectors.unpersist() } else { norms.unpersist() } - - model - } - - /** - * Implementation of K-Means algorithm. - */ - private def runAlgorithmWithWeight( - data: RDD[VectorWithNorm], - instr: Option[Instrumentation]): KMeansModel = { - - val sc = data.sparkContext - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString(this.distanceMeasure) - - val centers = initialModel match { - case Some(kMeansCenters) => - kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) - case None => - if (initializationMode == KMeans.RANDOM) { - initRandom(data) - } else { - initKMeansParallel(data, distanceMeasureInstance) - } - } - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - logInfo(f"Initialization with $initializationMode took $initTimeInSeconds%.3f seconds.") - - var converged = false - var cost = 0.0 - var iteration = 0 - - val iterationStartTime = System.nanoTime() - - instr.foreach(_.logNumFeatures(centers.head.vector.size)) - - // Execute iterations of Lloyd's algorithm until converged - while (iteration < maxIterations && !converged) { - val costAccum = sc.doubleAccumulator - val bcCenters = sc.broadcast(centers) - - // Find the new centers - val collected = data.mapPartitions { points => - val thisCenters = bcCenters.value - val dims = thisCenters.head.vector.size - - val sums = Array.fill(thisCenters.length)(Vectors.zeros(dims)) - - // clusterWeightSum is needed to calculate cluster center - // cluster center = - // sample1 * weight1/clusterWeightSum + sample2 * weight2/clusterWeightSum + ... - val clusterWeightSum = Array.ofDim[Double](thisCenters.length) - - points.foreach { point => - val (bestCenter, cost) = distanceMeasureInstance.findClosest(thisCenters, point) - costAccum.add(cost * point.weight) - distanceMeasureInstance.updateClusterSum(point, sums(bestCenter)) - clusterWeightSum(bestCenter) += point.weight - } - - clusterWeightSum.indices.filter(clusterWeightSum(_) > 0) - .map(j => (j, (sums(j), clusterWeightSum(j)))).iterator - }.reduceByKey { (sumweight1, sumweight2) => - axpy(1.0, sumweight2._1, sumweight1._1) - (sumweight1._1, sumweight1._2 + sumweight2._2) - }.collectAsMap() - - if (iteration == 0) { - instr.foreach(_.logNumExamples(costAccum.count)) - instr.foreach(_.logSumOfWeights(collected.values.map(_._2).sum)) - } - - val newCenters = collected.mapValues { case (sum, weightSum) => - distanceMeasureInstance.centroid(sum, weightSum) - } - - bcCenters.destroy() - - // Update the cluster centers and costs - converged = true - newCenters.foreach { case (j, newCenter) => - if (converged && - !distanceMeasureInstance.isCenterConverged(centers(j), newCenter, epsilon)) { - converged = false - } - centers(j) = newCenter - } - - cost = costAccum.value - iteration += 1 - } - - val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 - logInfo(f"Iterations took $iterationTimeInSeconds%.3f seconds.") - - if (iteration == maxIterations) { - logInfo(s"KMeans reached the max number of iterations: $maxIterations.") - } else { - logInfo(s"KMeans converged in $iteration iterations.") - } - - logInfo(s"The cost is $cost.") - - new KMeansModel(centers.map(_.vector), distanceMeasure, cost, iteration) - } - - /** - * Initialize a set of cluster centers at random. - */ - private[spark] def initRandom(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { - // Select without replacement; may still produce duplicates if the data has < k distinct - // points, so deduplicate the centroids to match the behavior of k-means|| in the same situation - data.takeSample(false, k, new XORShiftRandom(this.seed).nextInt()) - .map(_.vector).distinct.map(new VectorWithNorm(_)) - } - - /** - * Initialize a set of cluster centers using the k-means|| algorithm by Bahmani et al. - * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries - * to find dissimilar cluster centers by starting with a random center and then doing - * passes where more centers are chosen with probability proportional to their squared distance - * to the current cluster set. It results in a provable approximation to an optimal clustering. - * - * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. - */ - private[spark] def initKMeansParallel(data: RDD[VectorWithNorm], - distanceMeasureInstance: DistanceMeasure): Array[VectorWithNorm] = { - // Initialize empty centers and point costs. - var costs = data.map(_ => Double.PositiveInfinity) - - // Initialize the first center to a random point. - val seed = new XORShiftRandom(this.seed).nextInt() - val sample = data.takeSample(false, 1, seed) - // Could be empty if data is empty; fail with a better message early: - require(sample.nonEmpty, s"No samples available from $data") - - val centers = ArrayBuffer[VectorWithNorm]() - var newCenters = Seq(sample.head.toDense) - centers ++= newCenters - - // On each step, sample 2 * k points on average with probability proportional - // to their squared distance from the centers. Note that only distances between points - // and new centers are computed in each iteration. - var step = 0 - val bcNewCentersList = ArrayBuffer[Broadcast[_]]() - while (step < initializationSteps) { - val bcNewCenters = data.context.broadcast(newCenters) - bcNewCentersList += bcNewCenters - val preCosts = costs - costs = data.zip(preCosts).map { case (point, cost) => - math.min(distanceMeasureInstance.pointCost(bcNewCenters.value, point), cost) - }.persist(StorageLevel.MEMORY_AND_DISK) - val sumCosts = costs.sum() - - bcNewCenters.unpersist() - preCosts.unpersist() - - val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointCosts) => - val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - pointCosts.filter { case (_, c) => rand.nextDouble() < 2.0 * c * k / sumCosts }.map(_._1) - }.collect() - newCenters = chosen.map(_.toDense) - centers ++= newCenters - step += 1 - } - - costs.unpersist() - bcNewCentersList.foreach(_.destroy()) - - val distinctCenters = centers.map(_.vector).distinct.map(new VectorWithNorm(_)) - - if (distinctCenters.size <= k) { - distinctCenters.toArray - } else { - // Finally, we might have a set of more than k distinct candidate centers; weight each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick k of them - val bcCenters = data.context.broadcast(distinctCenters) - val countMap = data - .map(distanceMeasureInstance.findClosest(bcCenters.value, _)._1) - .countByValue() - - bcCenters.destroy() - - val myWeights = distinctCenters.indices.map(countMap.getOrElse(_, 0L).toDouble).toArray - LocalKMeans.kMeansPlusPlus(0, distinctCenters.toArray, myWeights, k, 30) - } - } -} - - -/** - * Top-level methods for calling K-means clustering. - */ -@Since("0.8.0") -object KMeans { - - // Initialization mode names - @Since("0.8.0") - val RANDOM = "random" - @Since("0.8.0") - val K_MEANS_PARALLEL = "k-means||" - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - * @param seed Random seed for cluster initialization. Default is to generate seed based - * on system time. - */ - @Since("2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - initializationMode: String, - seed: Long): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .setSeed(seed) - .run(data) - } - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - */ - @Since("2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - initializationMode: String): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .run(data) - } - - /** - * Trains a k-means model using specified parameters and the default values for unspecified. - */ - @Since("0.8.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .run(data) - } - - private[spark] def validateInitMode(initMode: String): Boolean = { - initMode match { - case KMeans.RANDOM => true - case KMeans.K_MEANS_PARALLEL => true - case _ => false - } - } -} - -/** - * A vector with its norm for fast distance computation. - */ -private[spark] class VectorWithNorm( - val vector: Vector, - val norm: Double, - val weight: Double = 1.0) extends Serializable { - - def this(vector: Vector) = this(vector, Vectors.norm(vector, 2.0)) - - def this(array: Array[Double]) = this(Vectors.dense(array)) - - /** Converts the vector to a dense vector. */ - def toDense: VectorWithNorm = new VectorWithNorm(Vectors.dense(vector.toArray), norm, weight) -} diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala deleted file mode 100644 index fa40a1000..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ /dev/null @@ -1,675 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.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.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) - - setDefault(smoothing -> 1.0, modelType -> NaiveBayes.Multinomial) -} - -// 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) - - /** - * 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) - - /** - * 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 sc = dataset.sparkSession.sparkContext - val model = if (Utils.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) - } - model - 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 spark = dataset.sparkSession - import spark.implicits._ - - val sc = spark.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"NaiveBayesDAL fit using $executor_num Executors") - - // 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) - - // 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 - } - - instr.logNumClasses(numClasses) - - val labeledPointsDS = dataset - .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) - - val model = new NaiveBayesDALImpl(uid, numClasses, - 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 - model.setOldLabels(labels) - - 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/spark-3.0.2/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/clustering/KMeans.scala deleted file mode 100644 index 27df400a8..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ /dev/null @@ -1,493 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.clustering - -import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.Since -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.ml.util._ -import org.apache.spark.ml.{Estimator, Model, PipelineStage} -import org.apache.spark.mllib.clustering.{DistanceMeasure, VectorWithNorm, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorVersion - -import scala.collection.mutable - - -/** - * Common params for KMeans and KMeansModel - */ -private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFeaturesCol - with HasSeed with HasPredictionCol with HasTol with HasDistanceMeasure with HasWeightCol { - - /** - * The number of clusters to create (k). Must be > 1. Note that it is possible for fewer than - * k clusters to be returned, for example, if there are fewer than k distinct points to cluster. - * Default: 2. - * @group param - */ - @Since("1.5.0") - final val k = new IntParam(this, "k", "The number of clusters to create. " + - "Must be > 1.", ParamValidators.gt(1)) - - /** @group getParam */ - @Since("1.5.0") - def getK: Int = $(k) - - /** - * Param for the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - * @group expertParam - */ - @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + - "Supported options: 'random' and 'k-means||'.", - (value: String) => MLlibKMeans.validateInitMode(value)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitMode: String = $(initMode) - - /** - * Param for the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Must be > 0. Default: 2. - * @group expertParam - */ - @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + - "initialization mode. Must be > 0.", ParamValidators.gt(0)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitSteps: Int = $(initSteps) - - setDefault(k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, initSteps -> 2, - tol -> 1e-4, distanceMeasure -> DistanceMeasure.EUCLIDEAN) - - /** - * Validates and transforms the input schema. - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.validateVectorCompatibleColumn(schema, getFeaturesCol) - SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) - } -} - -/** - * Model fitted by KMeans. - * - * @param parentModel a model trained by spark.mllib.clustering.KMeans. - */ -@Since("1.5.0") -class KMeansModel private[ml] ( - @Since("1.5.0") override val uid: String, - private[clustering] val parentModel: MLlibKMeansModel) - extends Model[KMeansModel] with KMeansParams with GeneralMLWritable - with HasTrainingSummary[KMeansSummary] { - - @Since("3.0.0") - lazy val numFeatures: Int = parentModel.clusterCenters.head.size - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeansModel = { - val copied = copyValues(new KMeansModel(uid, parentModel), extra) - copied.setSummary(trainingSummary).setParent(this.parent) - } - - /** @group setParam */ - @Since("2.0.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("2.0.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val predictUDF = udf((vector: Vector) => predict(vector)) - - dataset.withColumn($(predictionCol), - predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)), - outputSchema($(predictionCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(predictionCol).nonEmpty) { - outputSchema = SchemaUtils.updateNumValues(outputSchema, - $(predictionCol), parentModel.k) - } - outputSchema - } - - @Since("3.0.0") - def predict(features: Vector): Int = parentModel.predict(features) - - @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[KMeansModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"KMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + - s"numFeatures=$numFeatures" - } - - /** - * Gets summary of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("2.0.0") - override def summary: KMeansSummary = super.summary -} - -/** Helper class for storing model data */ -private case class ClusterData(clusterIdx: Int, clusterCenter: Vector) - - -/** A writer for KMeans that handles the "internal" (or default) format */ -private class InternalKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: cluster centers - val data: Array[ClusterData] = instance.clusterCenters.zipWithIndex.map { - case (center, idx) => - ClusterData(idx, center) - } - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(data).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for KMeans that handles the "pmml" format */ -private class PMMLKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - instance.parentModel.toPMML(sc, path) - } -} - - -@Since("1.6.0") -object KMeansModel extends MLReadable[KMeansModel] { - - @Since("1.6.0") - override def read: MLReader[KMeansModel] = new KMeansModelReader - - @Since("1.6.0") - override def load(path: String): KMeansModel = super.load(path) - - /** - * We store all cluster centers in a single row and use this class to store model data by - * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. - */ - private case class OldData(clusterCenters: Array[OldVector]) - - private class KMeansModelReader extends MLReader[KMeansModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[KMeansModel].getName - - override def load(path: String): KMeansModel = { - // Import implicits for Dataset Encoder - val sparkSession = super.sparkSession - import sparkSession.implicits._ - - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - - val clusterCenters = if (majorVersion(metadata.sparkVersion) >= 2) { - val data: Dataset[ClusterData] = sparkSession.read.parquet(dataPath).as[ClusterData] - data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) - } else { - // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. - sparkSession.read.parquet(dataPath).as[OldData].head().clusterCenters - } - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) - metadata.getAndSetParams(model) - model - } - } -} - -/** - * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. - * - * @see Bahmani et al., Scalable k-means++. - */ -@Since("1.5.0") -class KMeans @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeans = defaultCopy(extra) - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("kmeans")) - - /** @group setParam */ - @Since("1.5.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitMode(value: String): this.type = set(initMode, value) - - /** @group expertSetParam */ - @Since("2.4.0") - def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitSteps(value: Int): this.type = set(initSteps, value) - - /** @group setParam */ - @Since("1.5.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.5.0") - def setTol(value: Double): this.type = set(tol, value) - - /** @group setParam */ - @Since("1.5.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** - * 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("3.0.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): KMeansModel = instrumented { instr => - transformSchema(dataset.schema, logging = true) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, featuresCol, predictionCol, k, initMode, initSteps, distanceMeasure, - maxIter, seed, tol, weightCol) - - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - val handleWeight = isDefined(weightCol) && $(weightCol).nonEmpty - val w = if (handleWeight) { - col($(weightCol)).cast(DoubleType) - } else { - lit(1.0) - } - - val instances: RDD[(Vector, Double)] = dataset - .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w).rdd.map { - case Row(point: Vector, weight: Double) => (point, weight) - } - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - val useKMeansDAL = Utils.isOAPEnabled() && isPlatformSupported && - $(distanceMeasure) == "euclidean" && !handleWeight - - val model = if (useKMeansDAL) { - trainWithDAL(instances, handlePersistence) - } else { - trainWithML(instances, handlePersistence) - } - - val summary = new KMeansSummary( - model.transform(dataset), - $(predictionCol), - $(featuresCol), - $(k), - model.parentModel.numIter, - model.parentModel.trainingCost) - - model.setSummary(Some(summary)) - instr.logNamedValue("clusterSizes", summary.clusterSizes) - - model - } - - private def trainWithDAL(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - - val sc = instances.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"KMeansDAL fit using $executor_num Executors") - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString($(distanceMeasure)) - - // Use MLlibKMeans to initialize centers - val mllibKMeans = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - - val dataWithNorm = instances.map { - case (point: Vector, weight: Double) => new VectorWithNorm(point) - } - - // Cache for init - dataWithNorm.persist(StorageLevel.MEMORY_AND_DISK) - - val centersWithNorm = if ($(initMode) == "random") { - mllibKMeans.initRandom(dataWithNorm) - } else { - mllibKMeans.initKMeansParallel(dataWithNorm, distanceMeasureInstance) - } - - dataWithNorm.unpersist() - - val centers = centersWithNorm.map(_.vector) - - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - - val strInitMode = $(initMode) - logInfo(f"Initialization with $strInitMode took $initTimeInSeconds%.3f seconds.") - - if (handlePersistence) { - instances.persist(StorageLevel.MEMORY_AND_DISK) - } - - val inputData = instances.map { - case (point: Vector, weight: Double) => point - } - - val kmeansDAL = new KMeansDALImpl(getK, getMaxIter, getTol, - DistanceMeasure.EUCLIDEAN, centers, executor_num, executor_cores) - - val parentModel = kmeansDAL.train(inputData, Option(instr)) - - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - - if (handlePersistence) { - instances.unpersist() - } - - model - } - - private def trainWithML(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - val oldVectorInstances = instances.map { - case (point: Vector, weight: Double) => (OldVectors.fromML(point), weight) - } - val algo = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - val parentModel = algo.runWithWeight(oldVectorInstances, handlePersistence, Some(instr)) - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - model - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } -} - -@Since("1.6.0") -object KMeans extends DefaultParamsReadable[KMeans] { - - @Since("1.6.0") - override def load(path: String): KMeans = super.load(path) -} - -/** - * Summary of KMeans. - * - * @param predictions `DataFrame` produced by `KMeansModel.transform()`. - * @param predictionCol Name for column of predicted clusters in `predictions`. - * @param featuresCol Name for column of features in `predictions`. - * @param k Number of clusters. - * @param numIter Number of iterations. - * @param trainingCost K-means cost (sum of squared distances to the nearest centroid for all - * points in the training dataset). This is equivalent to sklearn's inertia. - */ -@Since("2.0.0") -class KMeansSummary private[clustering] ( - predictions: DataFrame, - predictionCol: String, - featuresCol: String, - k: Int, - numIter: Int, - @Since("2.4.0") val trainingCost: Double) - extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/feature/PCA.scala deleted file mode 100644 index 8ca5a9b5d..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/feature/PCA.scala +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.feature - -import org.apache.hadoop.fs.Path - -import org.apache.spark.annotation.Since -import org.apache.spark.ml._ -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util._ -import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} -import org.apache.spark.sql._ -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.VersionUtils.majorVersion - -/** - * Params for [[PCA]] and [[PCAModel]]. - */ -private[feature] trait PCAParams extends Params with HasInputCol with HasOutputCol { - - /** - * The number of principal components. - * @group param - */ - final val k: IntParam = new IntParam(this, "k", "the number of principal components (> 0)", - ParamValidators.gt(0)) - - /** @group getParam */ - def getK: Int = $(k) - - /** Validates and transforms the input schema. */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - SchemaUtils.updateAttributeGroupSize(schema, $(outputCol), $(k)) - } -} - -/** - * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` - * principal components. - */ -@Since("1.5.0") -class PCA @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("pca")) - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** - * Computes a [[PCAModel]] that contains the principal components of the input vectors. - */ - @Since("2.0.0") - override def fit(dataset: Dataset[_]): PCAModel = { - transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd - val inputVectors = input.map { - case Row(v: Vector) => v - } - - val numFeatures = inputVectors.first().size - require($(k) <= numFeatures, - s"source vector size $numFeatures must be no less than k=$k") - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - - // Call oneDAL Correlation PCA implementation when numFeatures < 65535 and fall back otherwise - val parentModel = if (numFeatures < 65535 && Utils.isOAPEnabled() && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val pca = new PCADALImpl(k = $(k), executor_num, executor_cores) - val pcaModel = pca.train(inputVectors) - pcaModel - } else { - val inputOldVectors = inputVectors.map { - case v: Vector => OldVectors.fromML(v) - } - val pca = new feature.PCA(k = $(k)) - val pcaModel = pca.fit(inputOldVectors) - pcaModel - } - copyValues(new PCAModel(uid, parentModel.pc.asML, parentModel.explainedVariance.asML) - .setParent(this)) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCA = defaultCopy(extra) -} - -@Since("1.6.0") -object PCA extends DefaultParamsReadable[PCA] { - - @Since("1.6.0") - override def load(path: String): PCA = super.load(path) -} - -/** - * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. - * - * @param pc A principal components Matrix. Each column is one principal component. - * @param explainedVariance A vector of proportions of variance explained by - * each principal component. - */ -@Since("1.5.0") -class PCAModel private[ml] ( - @Since("1.5.0") override val uid: String, - @Since("2.0.0") val pc: DenseMatrix, - @Since("2.0.0") val explainedVariance: DenseVector) - extends Model[PCAModel] with PCAParams with MLWritable { - - import PCAModel._ - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** - * Transform a vector by computed Principal Components. - * - * @note Vectors to be transformed must be the same length as the source vectors given - * to `PCA.fit()`. - */ - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val transposed = pc.transpose - val transformer = udf { vector: Vector => transposed.multiply(vector) } - dataset.withColumn($(outputCol), transformer(col($(inputCol))), - outputSchema($(outputCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(outputCol).nonEmpty) { - outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, - $(outputCol), $(k)) - } - outputSchema - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCAModel = { - val copied = new PCAModel(uid, pc, explainedVariance) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new PCAModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"PCAModel: uid=$uid, k=${$(k)}" - } -} - -@Since("1.6.0") -object PCAModel extends MLReadable[PCAModel] { - - private[PCAModel] class PCAModelWriter(instance: PCAModel) extends MLWriter { - - private case class Data(pc: DenseMatrix, explainedVariance: DenseVector) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.pc, instance.explainedVariance) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class PCAModelReader extends MLReader[PCAModel] { - - private val className = classOf[PCAModel].getName - - /** - * Loads a [[PCAModel]] from data located at the input path. Note that the model includes an - * `explainedVariance` member that is not recorded by Spark 1.6 and earlier. A model - * can be loaded from such older data but will have an empty vector for - * `explainedVariance`. - * - * @param path path to serialized model data - * @return a [[PCAModel]] - */ - override def load(path: String): PCAModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val model = if (majorVersion(metadata.sparkVersion) >= 2) { - val Row(pc: DenseMatrix, explainedVariance: DenseVector) = - sparkSession.read.parquet(dataPath) - .select("pc", "explainedVariance") - .head() - new PCAModel(metadata.uid, pc, explainedVariance) - } else { - // pc field is the old matrix format in Spark <= 1.6 - // explainedVariance field is not present in Spark <= 1.6 - val Row(pc: OldDenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() - new PCAModel(metadata.uid, pc.asML, new DenseVector(Array.emptyDoubleArray)) - } - metadata.getAndSetParams(model) - model - } - } - - @Since("1.6.0") - override def read: MLReader[PCAModel] = new PCAModelReader - - @Since("1.6.0") - override def load(path: String): PCAModel = super.load(path) -} diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/regression/LinearRegression.scala deleted file mode 100644 index 62e5e3283..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ /dev/null @@ -1,1059 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.regression - -import scala.collection.mutable - -import breeze.linalg.{DenseVector => BDV} -import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, LBFGSB => BreezeLBFGSB, OWLQN => BreezeOWLQN} -import breeze.stats.distributions.StudentsT -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkException -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PipelineStage, PredictorParams} -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{Vector, Vectors} -import org.apache.spark.ml.linalg.BLAS._ -import org.apache.spark.ml.optim.WeightedLeastSquares -import org.apache.spark.ml.optim.aggregator.{HuberAggregator, LeastSquaresAggregator} -import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} -import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.stat._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.regression.{LinearRegressionModel => OldLinearRegressionModel} -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, DoubleType, StructType} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorMinorVersion - -/** - * Params for linear regression. - */ -private[regression] trait LinearRegressionParams extends PredictorParams - with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol - with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver - with HasAggregationDepth with HasLoss { - - import LinearRegression._ - - /** - * The solver algorithm for optimization. - * Supported options: "l-bfgs", "normal" and "auto". - * Default: "auto" - * - * @group param - */ - @Since("1.6.0") - final override val solver: Param[String] = new Param[String](this, "solver", - "The solver algorithm for optimization. Supported options: " + - s"${supportedSolvers.mkString(", ")}. (Default auto)", - ParamValidators.inArray[String](supportedSolvers)) - - /** - * The loss function to be optimized. - * Supported options: "squaredError" and "huber". - * Default: "squaredError" - * - * @group param - */ - @Since("2.3.0") - final override val loss: Param[String] = new Param[String](this, "loss", "The loss function to" + - s" be optimized. Supported options: ${supportedLosses.mkString(", ")}. (Default squaredError)", - ParamValidators.inArray[String](supportedLosses)) - - /** - * The shape parameter to control the amount of robustness. Must be > 1.0. - * At larger values of epsilon, the huber criterion becomes more similar to least squares - * regression; for small values of epsilon, the criterion is more similar to L1 regression. - * Default is 1.35 to get as much robustness as possible while retaining - * 95% statistical efficiency for normally distributed data. It matches sklearn - * HuberRegressor and is "M" from - * A robust hybrid of lasso and ridge regression. - * Only valid when "loss" is "huber". - * - * @group expertParam - */ - @Since("2.3.0") - final val epsilon = new DoubleParam(this, "epsilon", "The shape parameter to control the " + - "amount of robustness. Must be > 1.0.", ParamValidators.gt(1.0)) - - /** @group getExpertParam */ - @Since("2.3.0") - def getEpsilon: Double = $(epsilon) - - setDefault(regParam -> 0.0, fitIntercept -> true, standardization -> true, - elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, solver -> Auto, - aggregationDepth -> 2, loss -> SquaredError, epsilon -> 1.35) - - override protected def validateAndTransformSchema( - schema: StructType, - fitting: Boolean, - featuresDataType: DataType): StructType = { - if (fitting) { - if ($(loss) == Huber) { - require($(solver)!= Normal, "LinearRegression with huber loss doesn't support " + - "normal solver, please change solver to auto or l-bfgs.") - require($(elasticNetParam) == 0.0, "LinearRegression with huber loss only supports " + - s"L2 regularization, but got elasticNetParam = $getElasticNetParam.") - } - } - super.validateAndTransformSchema(schema, fitting, featuresDataType) - } -} - -/** - * Linear regression. - * - * The learning objective is to minimize the specified loss function, with regularization. - * This supports two kinds of loss: - * - squaredError (a.k.a squared loss) - * - huber (a hybrid of squared error for relatively small errors and absolute error for - * relatively large ones, and we estimate the scale parameter from training data) - * - * This supports multiple types of regularization: - * - none (a.k.a. ordinary least squares) - * - L2 (ridge regression) - * - L1 (Lasso) - * - L2 + L1 (elastic net) - * - * The squared error objective function is: - * - *
- * $$ - * \begin{align} - * \min_{w}\frac{1}{2n}{\sum_{i=1}^n(X_{i}w - y_{i})^{2} + - * \lambda\left[\frac{1-\alpha}{2}{||w||_{2}}^{2} + \alpha{||w||_{1}}\right]} - * \end{align} - * $$ - *
- * - * The huber objective function is: - * - *
- * $$ - * \begin{align} - * \min_{w, \sigma}\frac{1}{2n}{\sum_{i=1}^n\left(\sigma + - * H_m\left(\frac{X_{i}w - y_{i}}{\sigma}\right)\sigma\right) + \frac{1}{2}\lambda {||w||_2}^2} - * \end{align} - * $$ - *
- * - * where - * - *
- * $$ - * \begin{align} - * H_m(z) = \begin{cases} - * z^2, & \text {if } |z| < \epsilon, \\ - * 2\epsilon|z| - \epsilon^2, & \text{otherwise} - * \end{cases} - * \end{align} - * $$ - *
- * - * Note: Fitting with huber loss only supports none and L2 regularization. - */ -@Since("1.3.0") -class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) - extends Regressor[Vector, LinearRegression, LinearRegressionModel] - with LinearRegressionParams with DefaultParamsWritable with Logging { - - import LinearRegression._ - - @Since("1.4.0") - def this() = this(Identifiable.randomUID("linReg")) - - /** - * Set the regularization parameter. - * Default is 0.0. - * - * @group setParam - */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - - /** - * Set if we should fit the intercept. - * Default is true. - * - * @group setParam - */ - @Since("1.5.0") - def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - - /** - * Whether to standardize the training features before fitting the model. - * The coefficients of models will be always returned on the original scale, - * so it will be transparent for users. - * Default is true. - * - * @note With/without standardization, the models should be always converged - * to the same solution when no regularization is applied. In R's GLMNET package, - * the default behavior is true as well. - * - * @group setParam - */ - @Since("1.5.0") - def setStandardization(value: Boolean): this.type = set(standardization, value) - - /** - * Set the ElasticNet mixing parameter. - * For alpha = 0, the penalty is an L2 penalty. - * For alpha = 1, it is an L1 penalty. - * For alpha in (0,1), the penalty is a combination of L1 and L2. - * Default is 0.0 which is an L2 penalty. - * - * Note: Fitting with huber loss only supports None and L2 regularization, - * so throws exception if this param is non-zero value. - * - * @group setParam - */ - @Since("1.4.0") - def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - - /** - * Set the maximum number of iterations. - * Default is 100. - * - * @group setParam - */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** - * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. - * Default is 1E-6. - * - * @group setParam - */ - @Since("1.4.0") - def setTol(value: Double): this.type = set(tol, value) - - /** - * Whether to over-/under-sample training instances according to the given weights in weightCol. - * If not set or empty, all instances are treated equally (weight 1.0). - * Default is not set, so all instances have weight one. - * - * @group setParam - */ - @Since("1.6.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - /** - * Set the solver algorithm used for optimization. - * In case of linear regression, this can be "l-bfgs", "normal" and "auto". - * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton - * optimization method. - * - "normal" denotes using Normal Equation as an analytical solution to the linear regression - * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. - * - "auto" (default) means that the solver algorithm is selected automatically. - * The Normal Equations solver will be used when possible, but this will automatically fall - * back to iterative optimization methods when needed. - * - * Note: Fitting with huber loss doesn't support normal solver, - * so throws exception if this param was set with "normal". - * @group setParam - */ - @Since("1.6.0") - def setSolver(value: String): this.type = set(solver, value) - - /** - * Suggested depth for treeAggregate (greater than or equal to 2). - * If the dimensions of features or the number of partitions are large, - * this param could be adjusted to a larger size. - * Default is 2. - * - * @group expertSetParam - */ - @Since("2.1.0") - def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - - /** - * Sets the value of param [[loss]]. - * Default is "squaredError". - * - * @group setParam - */ - @Since("2.3.0") - def setLoss(value: String): this.type = set(loss, value) - - /** - * Sets the value of param [[epsilon]]. - * Default is 1.35. - * - * @group setExpertParam - */ - @Since("2.3.0") - def setEpsilon(value: Double): this.type = set(epsilon, value) - - override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => - // Extract the number of features before deciding optimization solver. - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) - - val instances = extractInstances(dataset) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, solver, tol, - elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth, loss, - epsilon) - instr.logNumFeatures(numFeatures) - - if ($(loss) == SquaredError && (($(solver) == Auto && - numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == Normal)) { - // oneDAL only support simple linear regression and ridge regression - val paramSupported = ($(regParam) == 0) || ($(regParam) != 0 && $(elasticNetParam) == 0) - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - if (paramSupported && Utils.isOAPEnabled && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - logInfo(s"LinearRegressionDAL fit using $executor_num Executors") - - val optimizer = new LinearRegressionDALImpl($(fitIntercept), $(regParam), - elasticNetParam = $(elasticNetParam), $(standardization), true, - executor_num, executor_cores) - - // Return same model as WeightedLeastSquaresModel - val model = optimizer.train(dataset, Some(instr)) - - val lrModel = copyValues( - new LinearRegressionModel(uid, model.coefficients, model.intercept)) - - val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - summaryModel, - model.diagInvAtWA.toArray, - model.objectiveHistory) - - return lrModel.setSummary(Some(trainingSummary)) - } else { - // For low dimensional data, WeightedLeastSquares is more efficient since the - // training algorithm only requires one pass through the data. (SPARK-10668) - - val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), - elasticNetParam = $(elasticNetParam), $(standardization), true, - solverType = WeightedLeastSquares.Auto, maxIter = $(maxIter), tol = $(tol)) - val model = optimizer.fit(instances, instr = OptionalInstrumentation.create(instr)) - // When it is trained by WeightedLeastSquares, training summary does not - // attach returned model. - val lrModel = copyValues(new LinearRegressionModel(uid, model.coefficients, model.intercept)) - val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - summaryModel, - model.diagInvAtWA.toArray, - model.objectiveHistory) - - return lrModel.setSummary(Some(trainingSummary)) - } - } - - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - - val (featuresSummarizer, ySummarizer) = instances.treeAggregate( - (Summarizer.createSummarizerBuffer("mean", "std"), - Summarizer.createSummarizerBuffer("mean", "std", "count")))( - seqOp = (c: (SummarizerBuffer, SummarizerBuffer), instance: Instance) => - (c._1.add(instance.features, instance.weight), - c._2.add(Vectors.dense(instance.label), instance.weight)), - combOp = (c1: (SummarizerBuffer, SummarizerBuffer), - c2: (SummarizerBuffer, SummarizerBuffer)) => - (c1._1.merge(c2._1), c1._2.merge(c2._2)), - depth = $(aggregationDepth) - ) - - val yMean = ySummarizer.mean(0) - val rawYStd = ySummarizer.std(0) - - instr.logNumExamples(ySummarizer.count) - instr.logNamedValue(Instrumentation.loggerTags.meanOfLabels, yMean) - instr.logNamedValue(Instrumentation.loggerTags.varianceOfLabels, rawYStd) - instr.logSumOfWeights(featuresSummarizer.weightSum) - - if (rawYStd == 0.0) { - if ($(fitIntercept) || yMean == 0.0) { - // If the rawYStd==0 and fitIntercept==true, then the intercept is yMean with - // zero coefficient; as a result, training is not needed. - // Also, if yMean==0 and rawYStd==0, all the coefficients are zero regardless of - // the fitIntercept. - if (yMean == 0.0) { - instr.logWarning(s"Mean and standard deviation of the label are zero, so the " + - s"coefficients and the intercept will all be zero; as a result, training is not " + - s"needed.") - } else { - instr.logWarning(s"The standard deviation of the label is zero, so the coefficients " + - s"will be zeros and the intercept will be the mean of the label; as a result, " + - s"training is not needed.") - } - if (handlePersistence) instances.unpersist() - val coefficients = Vectors.sparse(numFeatures, Seq.empty) - val intercept = yMean - - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - model, - Array(0D), - Array(0D)) - - return model.setSummary(Some(trainingSummary)) - } else { - require($(regParam) == 0.0, "The standard deviation of the label is zero. " + - "Model cannot be regularized.") - instr.logWarning(s"The standard deviation of the label is zero. " + - "Consider setting fitIntercept=true.") - } - } - - // if y is constant (rawYStd is zero), then y cannot be scaled. In this case - // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. - val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) - val featuresMean = featuresSummarizer.mean.toArray - val featuresStd = featuresSummarizer.std.toArray - val bcFeaturesMean = instances.context.broadcast(featuresMean) - val bcFeaturesStd = instances.context.broadcast(featuresStd) - - if (!$(fitIntercept) && (0 until numFeatures).exists { i => - featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { - instr.logWarning("Fitting LinearRegressionModel without intercept on dataset with " + - "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + - "columns. This behavior is the same as R glmnet but different from LIBSVM.") - } - - // Since we implicitly do the feature scaling when we compute the cost function - // to improve the convergence, the effective regParam will be changed. - val effectiveRegParam = $(loss) match { - case SquaredError => $(regParam) / yStd - case Huber => $(regParam) - } - val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam - val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam - - val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 - val regularization = if (effectiveL2RegParam != 0.0) { - val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures - Some(new L2Regularization(effectiveL2RegParam, shouldApply, - if ($(standardization)) None else Some(getFeaturesStd))) - } else { - None - } - - val costFun = $(loss) match { - case SquaredError => - val getAggregatorFunc = new LeastSquaresAggregator(yStd, yMean, $(fitIntercept), - bcFeaturesStd, bcFeaturesMean)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) - case Huber => - val getAggregatorFunc = new HuberAggregator($(fitIntercept), $(epsilon), bcFeaturesStd)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) - } - - val optimizer = $(loss) match { - case SquaredError => - if ($(elasticNetParam) == 0.0 || effectiveRegParam == 0.0) { - new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) - } else { - val standardizationParam = $(standardization) - def effectiveL1RegFun = (index: Int) => { - if (standardizationParam) { - effectiveL1RegParam - } else { - // If `standardization` is false, we still standardize the data - // to improve the rate of convergence; as a result, we have to - // perform this reverse standardization by penalizing each component - // differently to get effectively the same objective function when - // the training dataset is not standardized. - if (featuresStd(index) != 0.0) effectiveL1RegParam / featuresStd(index) else 0.0 - } - } - new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, effectiveL1RegFun, $(tol)) - } - case Huber => - val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 - val lowerBounds = BDV[Double](Array.fill(dim)(Double.MinValue)) - // Optimize huber loss in space "\sigma > 0" - lowerBounds(dim - 1) = Double.MinPositiveValue - val upperBounds = BDV[Double](Array.fill(dim)(Double.MaxValue)) - new BreezeLBFGSB(lowerBounds, upperBounds, $(maxIter), 10, $(tol)) - } - - val initialValues = $(loss) match { - case SquaredError => - Vectors.zeros(numFeatures) - case Huber => - val dim = if ($(fitIntercept)) numFeatures + 2 else numFeatures + 1 - Vectors.dense(Array.fill(dim)(1.0)) - } - - val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialValues.asBreeze.toDenseVector) - - val (coefficients, intercept, scale, objectiveHistory) = { - /* - Note that in Linear Regression, the objective history (loss + regularization) returned - from optimizer is computed in the scaled space given by the following formula. -
- $$ - L &= 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 - + regTerms \\ - $$ -
- */ - val arrayBuilder = mutable.ArrayBuilder.make[Double] - var state: optimizer.State = null - while (states.hasNext) { - state = states.next() - arrayBuilder += state.adjustedValue - } - if (state == null) { - val msg = s"${optimizer.getClass.getName} failed." - instr.logError(msg) - throw new SparkException(msg) - } - - bcFeaturesMean.destroy() - bcFeaturesStd.destroy() - - val parameters = state.x.toArray.clone() - - /* - The coefficients are trained in the scaled space; we're converting them back to - the original space. - */ - val rawCoefficients: Array[Double] = $(loss) match { - case SquaredError => parameters - case Huber => parameters.slice(0, numFeatures) - } - - var i = 0 - val len = rawCoefficients.length - val multiplier = $(loss) match { - case SquaredError => yStd - case Huber => 1.0 - } - while (i < len) { - rawCoefficients(i) *= { if (featuresStd(i) != 0.0) multiplier / featuresStd(i) else 0.0 } - i += 1 - } - - val interceptValue: Double = if ($(fitIntercept)) { - $(loss) match { - case SquaredError => - /* - The intercept of squared error in R's GLMNET is computed using closed form - after the coefficients are converged. See the following discussion for detail. - http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet - */ - yMean - dot(Vectors.dense(rawCoefficients), Vectors.dense(featuresMean)) - case Huber => parameters(numFeatures) - } - } else { - 0.0 - } - - val scaleValue: Double = $(loss) match { - case SquaredError => 1.0 - case Huber => parameters.last - } - - (Vectors.dense(rawCoefficients).compressed, interceptValue, scaleValue, arrayBuilder.result()) - } - - if (handlePersistence) instances.unpersist() - - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept, scale)) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - $(featuresCol), - model, - Array(0D), - objectiveHistory) - - model.setSummary(Some(trainingSummary)) - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra) -} - -@Since("1.6.0") -object LinearRegression extends DefaultParamsReadable[LinearRegression] { - - @Since("1.6.0") - override def load(path: String): LinearRegression = super.load(path) - - /** - * When using `LinearRegression.solver` == "normal", the solver must limit the number of - * features to at most this number. The entire covariance matrix X^T^X will be collected - * to the driver. This limit helps prevent memory overflow errors. - */ - @Since("2.1.0") - val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES - - /** String name for "auto". */ - private[regression] val Auto = "auto" - - /** String name for "normal". */ - private[regression] val Normal = "normal" - - /** String name for "l-bfgs". */ - private[regression] val LBFGS = "l-bfgs" - - /** Set of solvers that LinearRegression supports. */ - private[regression] val supportedSolvers = Array(Auto, Normal, LBFGS) - - /** String name for "squaredError". */ - private[regression] val SquaredError = "squaredError" - - /** String name for "huber". */ - private[regression] val Huber = "huber" - - /** Set of loss function names that LinearRegression supports. */ - private[regression] val supportedLosses = Array(SquaredError, Huber) -} - -/** - * Model produced by [[LinearRegression]]. - */ -@Since("1.3.0") -class LinearRegressionModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("2.0.0") val coefficients: Vector, - @Since("1.3.0") val intercept: Double, - @Since("2.3.0") val scale: Double) - extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams with GeneralMLWritable - with HasTrainingSummary[LinearRegressionTrainingSummary] { - - private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = - this(uid, coefficients, intercept, 1.0) - - override val numFeatures: Int = coefficients.size - - /** - * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("1.5.0") - override def summary: LinearRegressionTrainingSummary = super.summary - - /** - * Evaluates the model on a test dataset. - * - * @param dataset Test dataset to evaluate model on. - */ - @Since("2.0.0") - def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() - new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, - $(labelCol), $(featuresCol), summaryModel, Array(0D)) - } - - /** - * If the prediction column is set returns the current model and prediction column, - * otherwise generates a new column and sets it as the prediction column on a new copy - * of the current model. - */ - private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { - $(predictionCol) match { - case "" => - val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString - (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) - case p => (this, p) - } - } - - - override def predict(features: Vector): Double = { - dot(features, coefficients) + intercept - } - - @Since("1.4.0") - override def copy(extra: ParamMap): LinearRegressionModel = { - val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) - newModel.setSummary(trainingSummary).setParent(parent) - } - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - * This also does not save the [[parent]] currently. - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" - } -} - -/** A writer for LinearRegression that handles the "internal" (or default) format */ -private class InternalLinearRegressionModelWriter - extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" - - private case class Data(intercept: Double, coefficients: Vector, scale: Double) - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[LinearRegressionModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: intercept, coefficients, scale - val data = Data(instance.intercept, instance.coefficients, instance.scale) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for LinearRegression that handles the "pmml" format */ -private class PMMLLinearRegressionModelWriter - extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - - override def stageName(): String = "org.apache.spark.ml.regression.LinearRegressionModel" - - private case class Data(intercept: Double, coefficients: Vector) - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val sc = sparkSession.sparkContext - // Construct the MLLib model which knows how to write to PMML. - val instance = stage.asInstanceOf[LinearRegressionModel] - val oldModel = new OldLinearRegressionModel(instance.coefficients, instance.intercept) - // Save PMML - oldModel.toPMML(sc, path) - } -} - -@Since("1.6.0") -object LinearRegressionModel extends MLReadable[LinearRegressionModel] { - - @Since("1.6.0") - override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader - - @Since("1.6.0") - override def load(path: String): LinearRegressionModel = super.load(path) - - private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[LinearRegressionModel].getName - - override def load(path: String): LinearRegressionModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.format("parquet").load(dataPath) - val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) - val model = if (majorVersion < 2 || (majorVersion == 2 && minorVersion <= 2)) { - // Spark 2.2 and before - val Row(intercept: Double, coefficients: Vector) = - MLUtils.convertVectorColumnsToML(data, "coefficients") - .select("intercept", "coefficients") - .head() - new LinearRegressionModel(metadata.uid, coefficients, intercept) - } else { - // Spark 2.3 and later - val Row(intercept: Double, coefficients: Vector, scale: Double) = - data.select("intercept", "coefficients", "scale").head() - new LinearRegressionModel(metadata.uid, coefficients, intercept, scale) - } - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Linear regression training results. Currently, the training summary ignores the - * training weights except for the objective trace. - * - * @param predictions predictions output by the model's `transform` method. - * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. - */ -@Since("1.5.0") -class LinearRegressionTrainingSummary private[regression] ( - predictions: DataFrame, - predictionCol: String, - labelCol: String, - featuresCol: String, - model: LinearRegressionModel, - diagInvAtWA: Array[Double], - val objectiveHistory: Array[Double]) - extends LinearRegressionSummary( - predictions, - predictionCol, - labelCol, - featuresCol, - model, - diagInvAtWA) { - - /** - * Number of training iterations until termination - * - * This value is only available when using the "l-bfgs" solver. - * - * @see `LinearRegression.solver` - */ - @Since("1.5.0") - val totalIterations = objectiveHistory.length - -} - -/** - * Linear regression results evaluated on a dataset. - * - * @param predictions predictions output by the model's `transform` method. - * @param predictionCol Field in "predictions" which gives the predicted value of the label at - * each instance. - * @param labelCol Field in "predictions" which gives the true label of each instance. - * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. - */ -@Since("1.5.0") -class LinearRegressionSummary private[regression] ( - @transient val predictions: DataFrame, - val predictionCol: String, - val labelCol: String, - val featuresCol: String, - private val privateModel: LinearRegressionModel, - private val diagInvAtWA: Array[Double]) extends Serializable { - - @transient private val metrics = new RegressionMetrics( - predictions - .select(col(predictionCol), col(labelCol).cast(DoubleType)) - .rdd - .map { case Row(pred: Double, label: Double) => (pred, label) }, - !privateModel.getFitIntercept) - - /** - * Returns the explained variance regression score. - * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) - * Reference: - * Wikipedia explain variation - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val explainedVariance: Double = metrics.explainedVariance - - /** - * Returns the mean absolute error, which is a risk function corresponding to the - * expected value of the absolute error loss or l1-norm loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val meanAbsoluteError: Double = metrics.meanAbsoluteError - - /** - * Returns the mean squared error, which is a risk function corresponding to the - * expected value of the squared error loss or quadratic loss. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val meanSquaredError: Double = metrics.meanSquaredError - - /** - * Returns the root mean squared error, which is defined as the square root of - * the mean squared error. - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val rootMeanSquaredError: Double = metrics.rootMeanSquaredError - - /** - * Returns R^2^, the coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("1.5.0") - val r2: Double = metrics.r2 - - /** - * Returns Adjusted R^2^, the adjusted coefficient of determination. - * Reference: - * Wikipedia coefficient of determination - * - * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. - * This will change in later Spark versions. - */ - @Since("2.3.0") - val r2adj: Double = { - val interceptDOF = if (privateModel.getFitIntercept) 1 else 0 - 1 - (1 - r2) * (numInstances - interceptDOF) / - (numInstances - privateModel.coefficients.size - interceptDOF) - } - - /** Residuals (label - predicted value) */ - @Since("1.5.0") - @transient lazy val residuals: DataFrame = { - val t = udf { (pred: Double, label: Double) => label - pred } - predictions.select(t(col(predictionCol), col(labelCol)).as("residuals")) - } - - /** Number of instances in DataFrame predictions */ - lazy val numInstances: Long = predictions.count() - - /** Degrees of freedom */ - @Since("2.2.0") - val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { - numInstances - privateModel.coefficients.size - 1 - } else { - numInstances - privateModel.coefficients.size - } - - /** - * The weighted residuals, the usual residuals rescaled by - * the square root of the instance weights. - */ - lazy val devianceResiduals: Array[Double] = { - val weighted = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - lit(1.0) - } else { - sqrt(col(privateModel.getWeightCol)) - } - val dr = predictions - .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) - .multiply(weighted).as("weightedResiduals")) - .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) - .first() - Array(dr.getDouble(0), dr.getDouble(1)) - } - - /** - * Standard error of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val coefficientStandardErrors: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No Std. Error of coefficients available for this LinearRegressionModel") - } else { - val rss = - if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { - meanSquaredError * numInstances - } else { - val t = udf { (pred: Double, label: Double, weight: Double) => - math.pow(label - pred, 2.0) * weight } - predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), - col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) - } - val sigma2 = rss / degreesOfFreedom - diagInvAtWA.map(_ * sigma2).map(math.sqrt) - } - } - - /** - * T-statistic of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val tValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No t-statistic available for this LinearRegressionModel") - } else { - val estimate = if (privateModel.getFitIntercept) { - Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) - } else { - privateModel.coefficients.toArray - } - estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } - } - } - - /** - * Two-sided p-value of estimated coefficients and intercept. - * This value is only available when using the "normal" solver. - * - * If `LinearRegression.fitIntercept` is set to true, - * then the last element returned corresponds to the intercept. - * - * @see `LinearRegression.solver` - */ - lazy val pValues: Array[Double] = { - if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { - throw new UnsupportedOperationException( - "No p-value available for this LinearRegressionModel") - } else { - tValues.map { x => 2.0 * (1.0 - StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) } - } - } - -} - diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/clustering/KMeans.scala deleted file mode 100644 index a0f050c65..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ /dev/null @@ -1,520 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.clustering - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.annotation.Since -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.Logging -import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS.axpy -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils -import org.apache.spark.util.random.XORShiftRandom - -/** - * K-means clustering with a k-means++ like initialization mode - * (the k-means|| algorithm by Bahmani et al). - * - * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given - * to it should be cached by the user. - */ -@Since("0.8.0") -class KMeans private ( - private var k: Int, - private var maxIterations: Int, - private var initializationMode: String, - private var initializationSteps: Int, - private var epsilon: Double, - private var seed: Long, - private var distanceMeasure: String) extends Serializable with Logging { - - @Since("0.8.0") - private def this(k: Int, maxIterations: Int, initializationMode: String, initializationSteps: Int, - epsilon: Double, seed: Long) = - this(k, maxIterations, initializationMode, initializationSteps, - epsilon, seed, DistanceMeasure.EUCLIDEAN) - - /** - * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, - * initializationMode: "k-means||", initializationSteps: 2, epsilon: 1e-4, seed: random, - * distanceMeasure: "euclidean"}. - */ - @Since("0.8.0") - def this() = this(2, 20, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong(), - DistanceMeasure.EUCLIDEAN) - - /** - * Number of clusters to create (k). - * - * @note It is possible for fewer than k clusters to - * be returned, for example, if there are fewer than k distinct points to cluster. - */ - @Since("1.4.0") - def getK: Int = k - - /** - * Set the number of clusters to create (k). - * - * @note It is possible for fewer than k clusters to - * be returned, for example, if there are fewer than k distinct points to cluster. Default: 2. - */ - @Since("0.8.0") - def setK(k: Int): this.type = { - require(k > 0, - s"Number of clusters must be positive but got ${k}") - this.k = k - this - } - - /** - * Maximum number of iterations allowed. - */ - @Since("1.4.0") - def getMaxIterations: Int = maxIterations - - /** - * Set maximum number of iterations allowed. Default: 20. - */ - @Since("0.8.0") - def setMaxIterations(maxIterations: Int): this.type = { - require(maxIterations >= 0, - s"Maximum of iterations must be nonnegative but got ${maxIterations}") - this.maxIterations = maxIterations - this - } - - /** - * The initialization algorithm. This can be either "random" or "k-means||". - */ - @Since("1.4.0") - def getInitializationMode: String = initializationMode - - /** - * Set the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - */ - @Since("0.8.0") - def setInitializationMode(initializationMode: String): this.type = { - KMeans.validateInitMode(initializationMode) - this.initializationMode = initializationMode - this - } - - /** - * Number of steps for the k-means|| initialization mode - */ - @Since("1.4.0") - def getInitializationSteps: Int = initializationSteps - - /** - * Set the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Default: 2. - */ - @Since("0.8.0") - def setInitializationSteps(initializationSteps: Int): this.type = { - require(initializationSteps > 0, - s"Number of initialization steps must be positive but got ${initializationSteps}") - this.initializationSteps = initializationSteps - this - } - - /** - * The distance threshold within which we've consider centers to have converged. - */ - @Since("1.4.0") - def getEpsilon: Double = epsilon - - /** - * Set the distance threshold within which we've consider centers to have converged. - * If all centers move less than this Euclidean distance, we stop iterating one run. - */ - @Since("0.8.0") - def setEpsilon(epsilon: Double): this.type = { - require(epsilon >= 0, - s"Distance threshold must be nonnegative but got ${epsilon}") - this.epsilon = epsilon - this - } - - /** - * The random seed for cluster initialization. - */ - @Since("1.4.0") - def getSeed: Long = seed - - /** - * Set the random seed for cluster initialization. - */ - @Since("1.4.0") - def setSeed(seed: Long): this.type = { - this.seed = seed - this - } - - /** - * The distance suite used by the algorithm. - */ - @Since("2.4.0") - def getDistanceMeasure: String = distanceMeasure - - /** - * Set the distance suite used by the algorithm. - */ - @Since("2.4.0") - def setDistanceMeasure(distanceMeasure: String): this.type = { - DistanceMeasure.validateDistanceMeasure(distanceMeasure) - this.distanceMeasure = distanceMeasure - this - } - - // Initial cluster centers can be provided as a KMeansModel object rather than using the - // random or k-means|| initializationMode - private var initialModel: Option[KMeansModel] = None - - /** - * Set the initial starting point, bypassing the random initialization or k-means|| - * The condition model.k == this.k must be met, failure results - * in an IllegalArgumentException. - */ - @Since("1.4.0") - def setInitialModel(model: KMeansModel): this.type = { - require(model.k == k, "mismatched cluster count") - initialModel = Some(model) - this - } - - /** - * Train a K-means model on the given set of points; `data` should be cached for high - * performance, because this is an iterative algorithm. - */ - @Since("0.8.0") - def run(data: RDD[Vector]): KMeansModel = { - val instances = data.map(point => (point, 1.0)) - val handlePersistence = data.getStorageLevel == StorageLevel.NONE - runWithWeight(instances, handlePersistence, None) - } - - private[spark] def runWithWeight( - instances: RDD[(Vector, Double)], - handlePersistence: Boolean, - instr: Option[Instrumentation]): KMeansModel = { - val norms = instances.map { case (v, _) => Vectors.norm(v, 2.0) } - val vectors = instances.zip(norms) - .map { case ((v, w), norm) => new VectorWithNorm(v, norm, w) } - - if (handlePersistence) { - vectors.persist(StorageLevel.MEMORY_AND_DISK) - } else { - // Compute squared norms and cache them. - norms.persist(StorageLevel.MEMORY_AND_DISK) - } - val model = runAlgorithmWithWeight(vectors, instr) - if (handlePersistence) { vectors.unpersist() } else { norms.unpersist() } - - model - } - - /** - * Implementation of K-Means algorithm. - */ - private def runAlgorithmWithWeight( - data: RDD[VectorWithNorm], - instr: Option[Instrumentation]): KMeansModel = { - - val sc = data.sparkContext - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString(this.distanceMeasure) - - val centers = initialModel match { - case Some(kMeansCenters) => - kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) - case None => - if (initializationMode == KMeans.RANDOM) { - initRandom(data) - } else { - initKMeansParallel(data, distanceMeasureInstance) - } - } - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - logInfo(f"Initialization with $initializationMode took $initTimeInSeconds%.3f seconds.") - - var converged = false - var cost = 0.0 - var iteration = 0 - - val iterationStartTime = System.nanoTime() - - instr.foreach(_.logNumFeatures(centers.head.vector.size)) - - // Execute iterations of Lloyd's algorithm until converged - while (iteration < maxIterations && !converged) { - val costAccum = sc.doubleAccumulator - val bcCenters = sc.broadcast(centers) - - // Find the new centers - val collected = data.mapPartitions { points => - val thisCenters = bcCenters.value - val dims = thisCenters.head.vector.size - - val sums = Array.fill(thisCenters.length)(Vectors.zeros(dims)) - - // clusterWeightSum is needed to calculate cluster center - // cluster center = - // sample1 * weight1/clusterWeightSum + sample2 * weight2/clusterWeightSum + ... - val clusterWeightSum = Array.ofDim[Double](thisCenters.length) - - points.foreach { point => - val (bestCenter, cost) = distanceMeasureInstance.findClosest(thisCenters, point) - costAccum.add(cost * point.weight) - distanceMeasureInstance.updateClusterSum(point, sums(bestCenter)) - clusterWeightSum(bestCenter) += point.weight - } - - clusterWeightSum.indices.filter(clusterWeightSum(_) > 0) - .map(j => (j, (sums(j), clusterWeightSum(j)))).iterator - }.reduceByKey { (sumweight1, sumweight2) => - axpy(1.0, sumweight2._1, sumweight1._1) - (sumweight1._1, sumweight1._2 + sumweight2._2) - }.collectAsMap() - - if (iteration == 0) { - instr.foreach(_.logNumExamples(costAccum.count)) - instr.foreach(_.logSumOfWeights(collected.values.map(_._2).sum)) - } - - val newCenters = collected.mapValues { case (sum, weightSum) => - distanceMeasureInstance.centroid(sum, weightSum) - } - - bcCenters.destroy() - - // Update the cluster centers and costs - converged = true - newCenters.foreach { case (j, newCenter) => - if (converged && - !distanceMeasureInstance.isCenterConverged(centers(j), newCenter, epsilon)) { - converged = false - } - centers(j) = newCenter - } - - cost = costAccum.value - iteration += 1 - } - - val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 - logInfo(f"Iterations took $iterationTimeInSeconds%.3f seconds.") - - if (iteration == maxIterations) { - logInfo(s"KMeans reached the max number of iterations: $maxIterations.") - } else { - logInfo(s"KMeans converged in $iteration iterations.") - } - - logInfo(s"The cost is $cost.") - - new KMeansModel(centers.map(_.vector), distanceMeasure, cost, iteration) - } - - /** - * Initialize a set of cluster centers at random. - */ - private[spark] def initRandom(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { - // Select without replacement; may still produce duplicates if the data has < k distinct - // points, so deduplicate the centroids to match the behavior of k-means|| in the same situation - data.takeSample(false, k, new XORShiftRandom(this.seed).nextInt()) - .map(_.vector).distinct.map(new VectorWithNorm(_)) - } - - /** - * Initialize a set of cluster centers using the k-means|| algorithm by Bahmani et al. - * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries - * to find dissimilar cluster centers by starting with a random center and then doing - * passes where more centers are chosen with probability proportional to their squared distance - * to the current cluster set. It results in a provable approximation to an optimal clustering. - * - * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. - */ - private[spark] def initKMeansParallel(data: RDD[VectorWithNorm], - distanceMeasureInstance: DistanceMeasure): Array[VectorWithNorm] = { - // Initialize empty centers and point costs. - var costs = data.map(_ => Double.PositiveInfinity) - - // Initialize the first center to a random point. - val seed = new XORShiftRandom(this.seed).nextInt() - val sample = data.takeSample(false, 1, seed) - // Could be empty if data is empty; fail with a better message early: - require(sample.nonEmpty, s"No samples available from $data") - - val centers = ArrayBuffer[VectorWithNorm]() - var newCenters = Seq(sample.head.toDense) - centers ++= newCenters - - // On each step, sample 2 * k points on average with probability proportional - // to their squared distance from the centers. Note that only distances between points - // and new centers are computed in each iteration. - var step = 0 - val bcNewCentersList = ArrayBuffer[Broadcast[_]]() - while (step < initializationSteps) { - val bcNewCenters = data.context.broadcast(newCenters) - bcNewCentersList += bcNewCenters - val preCosts = costs - costs = data.zip(preCosts).map { case (point, cost) => - math.min(distanceMeasureInstance.pointCost(bcNewCenters.value, point), cost) - }.persist(StorageLevel.MEMORY_AND_DISK) - val sumCosts = costs.sum() - - bcNewCenters.unpersist() - preCosts.unpersist() - - val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointCosts) => - val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - pointCosts.filter { case (_, c) => rand.nextDouble() < 2.0 * c * k / sumCosts }.map(_._1) - }.collect() - newCenters = chosen.map(_.toDense) - centers ++= newCenters - step += 1 - } - - costs.unpersist() - bcNewCentersList.foreach(_.destroy()) - - val distinctCenters = centers.map(_.vector).distinct.map(new VectorWithNorm(_)) - - if (distinctCenters.size <= k) { - distinctCenters.toArray - } else { - // Finally, we might have a set of more than k distinct candidate centers; weight each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick k of them - val bcCenters = data.context.broadcast(distinctCenters) - val countMap = data - .map(distanceMeasureInstance.findClosest(bcCenters.value, _)._1) - .countByValue() - - bcCenters.destroy() - - val myWeights = distinctCenters.indices.map(countMap.getOrElse(_, 0L).toDouble).toArray - LocalKMeans.kMeansPlusPlus(0, distinctCenters.toArray, myWeights, k, 30) - } - } -} - - -/** - * Top-level methods for calling K-means clustering. - */ -@Since("0.8.0") -object KMeans { - - // Initialization mode names - @Since("0.8.0") - val RANDOM = "random" - @Since("0.8.0") - val K_MEANS_PARALLEL = "k-means||" - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - * @param seed Random seed for cluster initialization. Default is to generate seed based - * on system time. - */ - @Since("2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - initializationMode: String, - seed: Long): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .setSeed(seed) - .run(data) - } - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - */ - @Since("2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - initializationMode: String): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .run(data) - } - - /** - * Trains a k-means model using specified parameters and the default values for unspecified. - */ - @Since("0.8.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .run(data) - } - - private[spark] def validateInitMode(initMode: String): Boolean = { - initMode match { - case KMeans.RANDOM => true - case KMeans.K_MEANS_PARALLEL => true - case _ => false - } - } -} - -/** - * A vector with its norm for fast distance computation. - */ -private[spark] class VectorWithNorm( - val vector: Vector, - val norm: Double, - val weight: Double = 1.0) extends Serializable { - - def this(vector: Vector) = this(vector, Vectors.norm(vector, 2.0)) - - def this(array: Array[Double]) = this(Vectors.dense(array)) - - /** Converts the vector to a dense vector. */ - def toDense: VectorWithNorm = new VectorWithNorm(Vectors.dense(vector.toArray), norm, weight) -} diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala deleted file mode 100644 index 961e0c9e5..000000000 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ /dev/null @@ -1,684 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.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.functions.checkNonNegativeWeight -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.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) - - setDefault(smoothing -> 1.0, modelType -> NaiveBayes.Multinomial) -} - -// 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) - - /** - * 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) - - /** - * 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 sc = dataset.sparkSession.sparkContext - val model = if (Utils.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) - } - model - 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 spark = dataset.sparkSession - import spark.implicits._ - - val sc = spark.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"NaiveBayesDAL fit using $executor_num Executors") - - // 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) - - // 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 - } - - instr.logNumClasses(numClasses) - - val labeledPointsDS = dataset - .select(col(getLabelCol), DatasetUtils.columnToVector(dataset, getFeaturesCol)) - - val model = new NaiveBayesDALImpl(uid, numClasses, - 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 - model.setOldLabels(labels) - - 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) { - checkNonNegativeWeight(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) { - checkNonNegativeWeight(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 = $(modelType) match { - case Bernoulli => - theta.map(value => value - math.log1p(-math.exp(value))) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + - "Variables thetaMinusNegTheta should only be precomputed in Bernoulli NB.") - } - - @transient private lazy val piMinusThetaSum = $(modelType) match { - case Bernoulli => - 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) - piMinusThetaSum - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + - "Variables piMinusThetaSum 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/spark-3.1.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala deleted file mode 100644 index 70f7c445f..000000000 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ /dev/null @@ -1,492 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.clustering - -import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.Since -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.ml.util._ -import org.apache.spark.ml.{Estimator, Model, PipelineStage} -import org.apache.spark.ml.functions.checkNonNegativeWeight -import org.apache.spark.mllib.clustering.{DistanceMeasure, VectorWithNorm, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.VersionUtils.majorVersion - -import scala.collection.mutable - - -/** - * Common params for KMeans and KMeansModel - */ -private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFeaturesCol - with HasSeed with HasPredictionCol with HasTol with HasDistanceMeasure with HasWeightCol { - - /** - * The number of clusters to create (k). Must be > 1. Note that it is possible for fewer than - * k clusters to be returned, for example, if there are fewer than k distinct points to cluster. - * Default: 2. - * @group param - */ - @Since("1.5.0") - final val k = new IntParam(this, "k", "The number of clusters to create. " + - "Must be > 1.", ParamValidators.gt(1)) - - /** @group getParam */ - @Since("1.5.0") - def getK: Int = $(k) - - /** - * Param for the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - * @group expertParam - */ - @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + - "Supported options: 'random' and 'k-means||'.", - (value: String) => MLlibKMeans.validateInitMode(value)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitMode: String = $(initMode) - - /** - * Param for the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 2 is almost always enough. Must be > 0. Default: 2. - * @group expertParam - */ - @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + - "initialization mode. Must be > 0.", ParamValidators.gt(0)) - - /** @group expertGetParam */ - @Since("1.5.0") - def getInitSteps: Int = $(initSteps) - - setDefault(k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, initSteps -> 2, - tol -> 1e-4, distanceMeasure -> DistanceMeasure.EUCLIDEAN) - - /** - * Validates and transforms the input schema. - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.validateVectorCompatibleColumn(schema, getFeaturesCol) - SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) - } -} - -/** - * Model fitted by KMeans. - * - * @param parentModel a model trained by spark.mllib.clustering.KMeans. - */ -@Since("1.5.0") -class KMeansModel private[ml] ( - @Since("1.5.0") override val uid: String, - private[clustering] val parentModel: MLlibKMeansModel) - extends Model[KMeansModel] with KMeansParams with GeneralMLWritable - with HasTrainingSummary[KMeansSummary] { - - @Since("3.0.0") - lazy val numFeatures: Int = parentModel.clusterCenters.head.size - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeansModel = { - val copied = copyValues(new KMeansModel(uid, parentModel), extra) - copied.setSummary(trainingSummary).setParent(this.parent) - } - - /** @group setParam */ - @Since("2.0.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("2.0.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val predictUDF = udf((vector: Vector) => predict(vector)) - - dataset.withColumn($(predictionCol), - predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)), - outputSchema($(predictionCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(predictionCol).nonEmpty) { - outputSchema = SchemaUtils.updateNumValues(outputSchema, - $(predictionCol), parentModel.k) - } - outputSchema - } - - @Since("3.0.0") - def predict(features: Vector): Int = parentModel.predict(features) - - @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) - - /** - * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. - * - * For [[KMeansModel]], this does NOT currently save the training [[summary]]. - * An option to save [[summary]] may be added in the future. - * - */ - @Since("1.6.0") - override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"KMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + - s"numFeatures=$numFeatures" - } - - /** - * Gets summary of model on training set. An exception is - * thrown if `hasSummary` is false. - */ - @Since("2.0.0") - override def summary: KMeansSummary = super.summary -} - -/** Helper class for storing model data */ -private case class ClusterData(clusterIdx: Int, clusterCenter: Vector) - - -/** A writer for KMeans that handles the "internal" (or default) format */ -private class InternalKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "internal" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - // Save metadata and Params - DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: cluster centers - val data: Array[ClusterData] = instance.clusterCenters.zipWithIndex.map { - case (center, idx) => - ClusterData(idx, center) - } - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(data).repartition(1).write.parquet(dataPath) - } -} - -/** A writer for KMeans that handles the "pmml" format */ -private class PMMLKMeansModelWriter extends MLWriterFormat with MLFormatRegister { - - override def format(): String = "pmml" - override def stageName(): String = "org.apache.spark.ml.clustering.KMeansModel" - - override def write(path: String, sparkSession: SparkSession, - optionMap: mutable.Map[String, String], stage: PipelineStage): Unit = { - val instance = stage.asInstanceOf[KMeansModel] - val sc = sparkSession.sparkContext - instance.parentModel.toPMML(sc, path) - } -} - - -@Since("1.6.0") -object KMeansModel extends MLReadable[KMeansModel] { - - @Since("1.6.0") - override def read: MLReader[KMeansModel] = new KMeansModelReader - - @Since("1.6.0") - override def load(path: String): KMeansModel = super.load(path) - - /** - * We store all cluster centers in a single row and use this class to store model data by - * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. - */ - private case class OldData(clusterCenters: Array[OldVector]) - - private class KMeansModelReader extends MLReader[KMeansModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[KMeansModel].getName - - override def load(path: String): KMeansModel = { - // Import implicits for Dataset Encoder - val sparkSession = super.sparkSession - import sparkSession.implicits._ - - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - - val clusterCenters = if (majorVersion(metadata.sparkVersion) >= 2) { - val data: Dataset[ClusterData] = sparkSession.read.parquet(dataPath).as[ClusterData] - data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) - } else { - // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. - sparkSession.read.parquet(dataPath).as[OldData].head().clusterCenters - } - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) - metadata.getAndSetParams(model) - model - } - } -} - -/** - * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. - * - * @see Bahmani et al., Scalable k-means++. - */ -@Since("1.5.0") -class KMeans @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { - - @Since("1.5.0") - override def copy(extra: ParamMap): KMeans = defaultCopy(extra) - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("kmeans")) - - /** @group setParam */ - @Since("1.5.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitMode(value: String): this.type = set(initMode, value) - - /** @group expertSetParam */ - @Since("2.4.0") - def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) - - /** @group expertSetParam */ - @Since("1.5.0") - def setInitSteps(value: Int): this.type = set(initSteps, value) - - /** @group setParam */ - @Since("1.5.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.5.0") - def setTol(value: Double): this.type = set(tol, value) - - /** @group setParam */ - @Since("1.5.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** - * 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("3.0.0") - def setWeightCol(value: String): this.type = set(weightCol, value) - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): KMeansModel = instrumented { instr => - transformSchema(dataset.schema, logging = true) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, featuresCol, predictionCol, k, initMode, initSteps, distanceMeasure, - maxIter, seed, tol, weightCol) - - val handleWeight = isDefined(weightCol) && $(weightCol).nonEmpty - val w = if (handleWeight) { - checkNonNegativeWeight(col($(weightCol)).cast(DoubleType)) - } else { - lit(1.0) - } - val instances = dataset.select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w) - .rdd.map { case Row(point: Vector, weight: Double) => (point, weight) } - - val handlePersistence = (dataset.storageLevel == StorageLevel.NONE) - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - val useKMeansDAL = Utils.isOAPEnabled() && isPlatformSupported && - $(distanceMeasure) == "euclidean" && !handleWeight - - val model = if (useKMeansDAL) { - trainWithDAL(instances, handlePersistence) - } else { - trainWithML(instances, handlePersistence) - } - - val summary = new KMeansSummary( - model.transform(dataset), - $(predictionCol), - $(featuresCol), - $(k), - model.parentModel.numIter, - model.parentModel.trainingCost) - - model.setSummary(Some(summary)) - instr.logNamedValue("clusterSizes", summary.clusterSizes) - - model - } - - private def trainWithDAL(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - - val sc = instances.sparkContext - - val executor_num = Utils.sparkExecutorNum(sc) - val executor_cores = Utils.sparkExecutorCores() - - logInfo(s"KMeansDAL fit using $executor_num Executors") - - val initStartTime = System.nanoTime() - - val distanceMeasureInstance = DistanceMeasure.decodeFromString($(distanceMeasure)) - - // Use MLlibKMeans to initialize centers - val mllibKMeans = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - - val dataWithNorm = instances.map { - case (point: Vector, weight: Double) => new VectorWithNorm(point) - } - - // Cache for init - dataWithNorm.persist(StorageLevel.MEMORY_AND_DISK) - - val centersWithNorm = if ($(initMode) == "random") { - mllibKMeans.initRandom(dataWithNorm) - } else { - mllibKMeans.initKMeansParallel(dataWithNorm, distanceMeasureInstance) - } - - dataWithNorm.unpersist() - - val centers = centersWithNorm.map(_.vector) - - val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - - val strInitMode = $(initMode) - logInfo(f"Initialization with $strInitMode took $initTimeInSeconds%.3f seconds.") - - if (handlePersistence) { - instances.persist(StorageLevel.MEMORY_AND_DISK) - } - - val inputData = instances.map { - case (point: Vector, weight: Double) => point - } - - val kmeansDAL = new KMeansDALImpl(getK, getMaxIter, getTol, - DistanceMeasure.EUCLIDEAN, centers, executor_num, executor_cores) - - val parentModel = kmeansDAL.train(inputData, Option(instr)) - - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - - if (handlePersistence) { - instances.unpersist() - } - - model - } - - private def trainWithML(instances: RDD[(Vector, Double)], - handlePersistence: Boolean): KMeansModel = instrumented { instr => - val oldVectorInstances = instances.map { - case (point: Vector, weight: Double) => (OldVectors.fromML(point), weight) - } - val algo = new MLlibKMeans() - .setK($(k)) - .setInitializationMode($(initMode)) - .setInitializationSteps($(initSteps)) - .setMaxIterations($(maxIter)) - .setSeed($(seed)) - .setEpsilon($(tol)) - .setDistanceMeasure($(distanceMeasure)) - val parentModel = algo.runWithWeight(oldVectorInstances, handlePersistence, Some(instr)) - val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) - model - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } -} - -@Since("1.6.0") -object KMeans extends DefaultParamsReadable[KMeans] { - - @Since("1.6.0") - override def load(path: String): KMeans = super.load(path) -} - -/** - * Summary of KMeans. - * - * @param predictions `DataFrame` produced by `KMeansModel.transform()`. - * @param predictionCol Name for column of predicted clusters in `predictions`. - * @param featuresCol Name for column of features in `predictions`. - * @param k Number of clusters. - * @param numIter Number of iterations. - * @param trainingCost K-means cost (sum of squared distances to the nearest centroid for all - * points in the training dataset). This is equivalent to sklearn's inertia. - */ -@Since("2.0.0") -class KMeansSummary private[clustering] ( - predictions: DataFrame, - predictionCol: String, - featuresCol: String, - k: Int, - numIter: Int, - @Since("2.4.0") val trainingCost: Double) - extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/feature/PCA.scala deleted file mode 100644 index 8ca5a9b5d..000000000 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/feature/PCA.scala +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.feature - -import org.apache.hadoop.fs.Path - -import org.apache.spark.annotation.Since -import org.apache.spark.ml._ -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util._ -import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} -import org.apache.spark.sql._ -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.VersionUtils.majorVersion - -/** - * Params for [[PCA]] and [[PCAModel]]. - */ -private[feature] trait PCAParams extends Params with HasInputCol with HasOutputCol { - - /** - * The number of principal components. - * @group param - */ - final val k: IntParam = new IntParam(this, "k", "the number of principal components (> 0)", - ParamValidators.gt(0)) - - /** @group getParam */ - def getK: Int = $(k) - - /** Validates and transforms the input schema. */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - SchemaUtils.updateAttributeGroupSize(schema, $(outputCol), $(k)) - } -} - -/** - * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` - * principal components. - */ -@Since("1.5.0") -class PCA @Since("1.5.0") ( - @Since("1.5.0") override val uid: String) - extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { - - @Since("1.5.0") - def this() = this(Identifiable.randomUID("pca")) - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setK(value: Int): this.type = set(k, value) - - /** - * Computes a [[PCAModel]] that contains the principal components of the input vectors. - */ - @Since("2.0.0") - override def fit(dataset: Dataset[_]): PCAModel = { - transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd - val inputVectors = input.map { - case Row(v: Vector) => v - } - - val numFeatures = inputVectors.first().size - require($(k) <= numFeatures, - s"source vector size $numFeatures must be no less than k=$k") - - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - dataset.sparkSession.sparkContext) - - // Call oneDAL Correlation PCA implementation when numFeatures < 65535 and fall back otherwise - val parentModel = if (numFeatures < 65535 && Utils.isOAPEnabled() && isPlatformSupported) { - val executor_num = Utils.sparkExecutorNum(dataset.sparkSession.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val pca = new PCADALImpl(k = $(k), executor_num, executor_cores) - val pcaModel = pca.train(inputVectors) - pcaModel - } else { - val inputOldVectors = inputVectors.map { - case v: Vector => OldVectors.fromML(v) - } - val pca = new feature.PCA(k = $(k)) - val pcaModel = pca.fit(inputOldVectors) - pcaModel - } - copyValues(new PCAModel(uid, parentModel.pc.asML, parentModel.explainedVariance.asML) - .setParent(this)) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCA = defaultCopy(extra) -} - -@Since("1.6.0") -object PCA extends DefaultParamsReadable[PCA] { - - @Since("1.6.0") - override def load(path: String): PCA = super.load(path) -} - -/** - * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. - * - * @param pc A principal components Matrix. Each column is one principal component. - * @param explainedVariance A vector of proportions of variance explained by - * each principal component. - */ -@Since("1.5.0") -class PCAModel private[ml] ( - @Since("1.5.0") override val uid: String, - @Since("2.0.0") val pc: DenseMatrix, - @Since("2.0.0") val explainedVariance: DenseVector) - extends Model[PCAModel] with PCAParams with MLWritable { - - import PCAModel._ - - /** @group setParam */ - @Since("1.5.0") - def setInputCol(value: String): this.type = set(inputCol, value) - - /** @group setParam */ - @Since("1.5.0") - def setOutputCol(value: String): this.type = set(outputCol, value) - - /** - * Transform a vector by computed Principal Components. - * - * @note Vectors to be transformed must be the same length as the source vectors given - * to `PCA.fit()`. - */ - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val outputSchema = transformSchema(dataset.schema, logging = true) - - val transposed = pc.transpose - val transformer = udf { vector: Vector => transposed.multiply(vector) } - dataset.withColumn($(outputCol), transformer(col($(inputCol))), - outputSchema($(outputCol)).metadata) - } - - @Since("1.5.0") - override def transformSchema(schema: StructType): StructType = { - var outputSchema = validateAndTransformSchema(schema) - if ($(outputCol).nonEmpty) { - outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, - $(outputCol), $(k)) - } - outputSchema - } - - @Since("1.5.0") - override def copy(extra: ParamMap): PCAModel = { - val copied = new PCAModel(uid, pc, explainedVariance) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new PCAModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"PCAModel: uid=$uid, k=${$(k)}" - } -} - -@Since("1.6.0") -object PCAModel extends MLReadable[PCAModel] { - - private[PCAModel] class PCAModelWriter(instance: PCAModel) extends MLWriter { - - private case class Data(pc: DenseMatrix, explainedVariance: DenseVector) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.pc, instance.explainedVariance) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class PCAModelReader extends MLReader[PCAModel] { - - private val className = classOf[PCAModel].getName - - /** - * Loads a [[PCAModel]] from data located at the input path. Note that the model includes an - * `explainedVariance` member that is not recorded by Spark 1.6 and earlier. A model - * can be loaded from such older data but will have an empty vector for - * `explainedVariance`. - * - * @param path path to serialized model data - * @return a [[PCAModel]] - */ - override def load(path: String): PCAModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - - val dataPath = new Path(path, "data").toString - val model = if (majorVersion(metadata.sparkVersion) >= 2) { - val Row(pc: DenseMatrix, explainedVariance: DenseVector) = - sparkSession.read.parquet(dataPath) - .select("pc", "explainedVariance") - .head() - new PCAModel(metadata.uid, pc, explainedVariance) - } else { - // pc field is the old matrix format in Spark <= 1.6 - // explainedVariance field is not present in Spark <= 1.6 - val Row(pc: OldDenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() - new PCAModel(metadata.uid, pc.asML, new DenseVector(Array.emptyDoubleArray)) - } - metadata.getAndSetParams(model) - model - } - } - - @Since("1.6.0") - override def read: MLReader[PCAModel] = new PCAModelReader - - @Since("1.6.0") - override def load(path: String): PCAModel = super.load(path) -} diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala deleted file mode 100644 index 71195397b..000000000 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ /dev/null @@ -1,1885 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.recommendation - -import java.{util => ju} -import java.io.IOException -import java.util.Locale - -import scala.collection.mutable -import scala.reflect.ClassTag -import scala.util.{Sorting, Try} -import scala.util.hashing.byteswap64 - -import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.hadoop.fs.Path -import org.json4s.DefaultFormats -import org.json4s.JsonDSL._ - -import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.linalg.BLAS -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Utils => DALImplUtils, _} -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.linalg.CholeskyDecomposition -import org.apache.spark.mllib.optimization.NNLS -import org.apache.spark.rdd.{DeterministicLevel, RDD} -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types._ -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils} -import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} -import org.apache.spark.util.random.XORShiftRandom - -/** - * Common params for ALS and ALSModel. - */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol - with HasBlockSize { - /** - * Param for the column name for user ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "user" - * @group param - */ - val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getUserCol: String = $(userCol) - - /** - * Param for the column name for item ids. Ids must be integers. Other - * numeric types are supported for this column, but will be cast to integers as long as they - * fall within the integer value range. - * Default: "item" - * @group param - */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + - "the integer value range.") - - /** @group getParam */ - def getItemCol: String = $(itemCol) - - /** - * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is - * out of integer range or contains a fractional part. - */ - protected[recommendation] val checkedCast = udf { (n: Any) => - n match { - case v: Int => v // Avoid unnecessary casting - case v: Number => - val intV = v.intValue - // Checks if number within Int range and has no fractional part. - if (v.doubleValue == intV) { - intV - } else { - throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"and without fractional part for columns ${$(userCol)} and ${$(itemCol)}. " + - s"Value $n was either out of Integer range or contained a fractional part that " + - s"could not be converted.") - } - case _ => throw new IllegalArgumentException(s"ALS only supports values in Integer range " + - s"for columns ${$(userCol)} and ${$(itemCol)}. Value $n was not numeric.") - } - } - - /** - * Param for strategy for dealing with unknown or new users/items at prediction time. - * This may be useful in cross-validation or production scenarios, for handling user/item ids - * the model has not seen in the training data. - * Supported values: - * - "nan": predicted value for unknown ids will be NaN. - * - "drop": rows in the input DataFrame containing unknown ids will be dropped from - * the output DataFrame containing predictions. - * Default: "nan". - * @group expertParam - */ - val coldStartStrategy = new Param[String](this, "coldStartStrategy", - "strategy for dealing with unknown or new users/items at prediction time. This may be " + - "useful in cross-validation or production scenarios, for handling user/item ids the model " + - "has not seen in the training data. Supported values: " + - s"${ALSModel.supportedColdStartStrategies.mkString(",")}.", - (s: String) => - ALSModel.supportedColdStartStrategies.contains(s.toLowerCase(Locale.ROOT))) - - /** @group expertGetParam */ - def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) - - setDefault(blockSize -> 4096) -} - -/** - * Common params for ALS. - */ -private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter with HasRegParam - with HasCheckpointInterval with HasSeed { - - /** - * Param for rank of the matrix factorization (positive). - * Default: 10 - * @group param - */ - val rank = new IntParam(this, "rank", "rank of the factorization", ParamValidators.gtEq(1)) - - /** @group getParam */ - def getRank: Int = $(rank) - - /** - * Param for number of user blocks (positive). - * Default: 10 - * @group param - */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumUserBlocks: Int = $(numUserBlocks) - - /** - * Param for number of item blocks (positive). - * Default: 10 - * @group param - */ - val numItemBlocks = new IntParam(this, "numItemBlocks", "number of item blocks", - ParamValidators.gtEq(1)) - - /** @group getParam */ - def getNumItemBlocks: Int = $(numItemBlocks) - - /** - * Param to decide whether to use implicit preference. - * Default: false - * @group param - */ - val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") - - /** @group getParam */ - def getImplicitPrefs: Boolean = $(implicitPrefs) - - /** - * Param for the alpha parameter in the implicit preference formulation (nonnegative). - * Default: 1.0 - * @group param - */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", - ParamValidators.gtEq(0)) - - /** @group getParam */ - def getAlpha: Double = $(alpha) - - /** - * Param for the column name for ratings. - * Default: "rating" - * @group param - */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") - - /** @group getParam */ - def getRatingCol: String = $(ratingCol) - - /** - * Param for whether to apply nonnegativity constraints. - * Default: false - * @group param - */ - val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares") - - /** @group getParam */ - def getNonnegative: Boolean = $(nonnegative) - - /** - * Param for StorageLevel for intermediate datasets. Pass in a string representation of - * `StorageLevel`. Cannot be "NONE". - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") - - /** @group expertGetParam */ - def getIntermediateStorageLevel: String = $(intermediateStorageLevel) - - /** - * Param for StorageLevel for ALS model factors. Pass in a string representation of - * `StorageLevel`. - * Default: "MEMORY_AND_DISK". - * - * @group expertParam - */ - val finalStorageLevel = new Param[String](this, "finalStorageLevel", - "StorageLevel for ALS model factors.", - (s: String) => Try(StorageLevel.fromString(s)).isSuccess) - - /** @group expertGetParam */ - def getFinalStorageLevel: String = $(finalStorageLevel) - - setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, - implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", - ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10, - intermediateStorageLevel -> "MEMORY_AND_DISK", finalStorageLevel -> "MEMORY_AND_DISK", - coldStartStrategy -> "nan") - - /** - * Validates and transforms the input schema. - * - * @param schema input schema - * @return output schema - */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - // rating will be cast to Float - SchemaUtils.checkNumericType(schema, $(ratingCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } -} - -/** - * Model fitted by ALS. - * - * @param rank rank of the matrix factorization model - * @param userFactors a DataFrame that stores user factors in two columns: `id` and `features` - * @param itemFactors a DataFrame that stores item factors in two columns: `id` and `features` - */ -@Since("1.3.0") -class ALSModel private[ml] ( - @Since("1.4.0") override val uid: String, - @Since("1.4.0") val rank: Int, - @transient val userFactors: DataFrame, - @transient val itemFactors: DataFrame) - extends Model[ALSModel] with ALSModelParams with MLWritable { - - /** @group setParam */ - @Since("1.4.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.4.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => - if (featuresA != null && featuresB != null) { - var dotProduct = 0.0f - var i = 0 - while (i < rank) { - dotProduct += featuresA(i) * featuresB(i) - i += 1 - } - dotProduct - } else { - Float.NaN - } - } - - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema) - // create a new column named map(predictionCol) by running the predict UDF. - val predictions = dataset - .join(userFactors, - checkedCast(dataset($(userCol))) === userFactors("id"), "left") - .join(itemFactors, - checkedCast(dataset($(itemCol))) === itemFactors("id"), "left") - .select(dataset("*"), - predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) - getColdStartStrategy match { - case ALSModel.Drop => - predictions.na.drop("all", Seq($(predictionCol))) - case ALSModel.NaN => - predictions - } - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - // user and item will be cast to Int - SchemaUtils.checkNumericType(schema, $(userCol)) - SchemaUtils.checkNumericType(schema, $(itemCol)) - SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALSModel = { - val copied = new ALSModel(uid, rank, userFactors, itemFactors) - copyValues(copied, extra).setParent(parent) - } - - @Since("1.6.0") - override def write: MLWriter = new ALSModel.ALSModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"ALSModel: uid=$uid, rank=$rank" - } - - /** - * Returns top `numItems` items recommended for each user, for all users. - * @param numItems max number of recommendations for each user - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numItems` items recommended for each user id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of user ids. The column name must match `userCol`. - * @param numItems max number of recommendations for each user. - * @return a DataFrame of (userCol: Int, recommendations), where recommendations are - * stored as an array of (itemCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item, for all items. - * @param numUsers max number of recommendations for each item - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.2.0") - def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns top `numUsers` users recommended for each item id in the input data set. Note that if - * there are duplicate ids in the input dataset, only one set of recommendations per unique id - * will be returned. - * @param dataset a Dataset containing a column of item ids. The column name must match `itemCol`. - * @param numUsers max number of recommendations for each item. - * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are - * stored as an array of (userCol: Int, rating: Float) Rows. - */ - @Since("2.3.0") - def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { - val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) - } - - /** - * Returns a subset of a factor DataFrame limited to only those unique ids contained - * in the input dataset. - * @param dataset input Dataset containing id column to user to filter factors. - * @param factors factor DataFrame to filter. - * @param column column name containing the ids in the input dataset. - * @return DataFrame containing factors only for those ids present in both the input dataset and - * the factor DataFrame. - */ - private def getSourceFactorSubset( - dataset: Dataset[_], - factors: DataFrame, - column: String): DataFrame = { - factors - .join(dataset.select(column), factors("id") === dataset(column), joinType = "left_semi") - .select(factors("id"), factors("features")) - } - - /** - * Makes recommendations for all users (or items). - * - * Note: the previous approach used for computing top-k recommendations - * used a cross-join followed by predicting a score for each row of the joined dataset. - * However, this results in exploding the size of intermediate data. While Spark SQL makes it - * relatively efficient, the approach implemented here is significantly more efficient. - * - * This approach groups factors into blocks and computes the top-k elements per block, - * using dot product and an efficient [[BoundedPriorityQueue]] (instead of gemm). - * It then computes the global top-k by aggregating the per block top-k elements with - * a [[TopByKeyAggregator]]. This significantly reduces the size of intermediate and shuffle data. - * This is the DataFrame equivalent to the approach used in - * [[org.apache.spark.mllib.recommendation.MatrixFactorizationModel]]. - * - * @param srcFactors src factors for which to generate recommendations - * @param dstFactors dst factors used to make recommendations - * @param srcOutputColumn name of the column for the source ID in the output DataFrame - * @param dstOutputColumn name of the column for the destination ID in the output DataFrame - * @param num max number of recommendations for each record - * @return a DataFrame of (srcOutputColumn: Int, recommendations), where recommendations are - * stored as an array of (dstOutputColumn: Int, rating: Float) Rows. - */ - private def recommendForAll( - srcFactors: DataFrame, - dstFactors: DataFrame, - srcOutputColumn: String, - dstOutputColumn: String, - num: Int, - blockSize: Int): DataFrame = { - import srcFactors.sparkSession.implicits._ - - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) - val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) - .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] - .flatMap { case (srcIter, dstIter) => - val m = srcIter.size - val n = math.min(dstIter.size, num) - val output = new Array[(Int, Int, Float)](m * n) - var i = 0 - val pq = new BoundedPriorityQueue[(Int, Float)](num)(Ordering.by(_._2)) - srcIter.foreach { case (srcId, srcFactor) => - dstIter.foreach { case (dstId, dstFactor) => - // We use F2jBLAS which is faster than a call to native BLAS for vector dot product - val score = BLAS.f2jBLAS.sdot(rank, srcFactor, 1, dstFactor, 1) - pq += dstId -> score - } - pq.foreach { case (dstId, score) => - output(i) = (srcId, dstId, score) - i += 1 - } - pq.clear() - } - output.toSeq - } - // We'll force the IDs to be Int. Unfortunately this converts IDs to Int in the output. - val topKAggregator = new TopByKeyAggregator[Int, Int, Float](num, Ordering.by(_._2)) - val recs = ratings.as[(Int, Int, Float)].groupByKey(_._1).agg(topKAggregator.toColumn) - .toDF("id", "recommendations") - - val arrayType = ArrayType( - new StructType() - .add(dstOutputColumn, IntegerType) - .add("rating", FloatType) - ) - recs.select($"id".as(srcOutputColumn), $"recommendations".cast(arrayType)) - } - - /** - * Blockifies factors to improve the efficiency of cross join - */ - private def blockify( - factors: Dataset[(Int, Array[Float])], - blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { - import factors.sparkSession.implicits._ - factors.mapPartitions(_.grouped(blockSize)) - } - -} - -@Since("1.6.0") -object ALSModel extends MLReadable[ALSModel] { - - private val NaN = "nan" - private val Drop = "drop" - private[recommendation] final val supportedColdStartStrategies = Array(NaN, Drop) - - @Since("1.6.0") - override def read: MLReader[ALSModel] = new ALSModelReader - - @Since("1.6.0") - override def load(path: String): ALSModel = super.load(path) - - private[ALSModel] class ALSModelWriter(instance: ALSModel) extends MLWriter { - - override protected def saveImpl(path: String): Unit = { - val extraMetadata = "rank" -> instance.rank - DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) - val userPath = new Path(path, "userFactors").toString - instance.userFactors.write.format("parquet").save(userPath) - val itemPath = new Path(path, "itemFactors").toString - instance.itemFactors.write.format("parquet").save(itemPath) - } - } - - private class ALSModelReader extends MLReader[ALSModel] { - - /** Checked against metadata when loading model */ - private val className = classOf[ALSModel].getName - - override def load(path: String): ALSModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - implicit val format = DefaultFormats - val rank = (metadata.metadata \ "rank").extract[Int] - val userPath = new Path(path, "userFactors").toString - val userFactors = sparkSession.read.format("parquet").load(userPath) - val itemPath = new Path(path, "itemFactors").toString - val itemFactors = sparkSession.read.format("parquet").load(itemPath) - - val model = new ALSModel(metadata.uid, rank, userFactors, itemFactors) - - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Alternating Least Squares (ALS) matrix factorization. - * - * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices, - * `X` and `Y`, i.e. `X * Yt = R`. Typically these approximations are called 'factor' matrices. - * The general approach is iterative. During each iteration, one of the factor matrices is held - * constant, while the other is solved for using least squares. The newly-solved factor matrix is - * then held constant while solving for the other factor matrix. - * - * This is a blocked implementation of the ALS factorization algorithm that groups the two sets - * of factors (referred to as "users" and "products") into blocks and reduces communication by only - * sending one copy of each user vector to each product block on each iteration, and only for the - * product blocks that need that user's feature vector. This is achieved by pre-computing some - * information about the ratings matrix to determine the "out-links" of each user (which blocks of - * products it will contribute to) and "in-link" information for each product (which of the feature - * vectors it receives from each user block it will depend on). This allows us to send only an - * array of feature vectors between each user block and product block, and have the product block - * find the users' ratings and update the products based on these messages. - * - * For implicit preference data, the algorithm used is based on - * "Collaborative Filtering for Implicit Feedback Datasets", available at - * https://doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. - * - * Essentially instead of finding the low-rank approximations to the rating matrix `R`, - * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if - * r is greater than 0 and 0 if r is less than or equal to 0. The ratings then act as 'confidence' - * values related to strength of indicated user - * preferences rather than explicit ratings given to items. - * - * Note: the input rating dataset to the ALS implementation should be deterministic. - * Nondeterministic data can cause failure during fitting ALS model. - * For example, an order-sensitive operation like sampling after a repartition makes dataset - * output nondeterministic, like `dataset.repartition(2).sample(false, 0.5, 1618)`. - * Checkpointing sampled dataset or adding a sort before sampling can help make the dataset - * deterministic. - */ -@Since("1.3.0") -class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams - with DefaultParamsWritable { - - import org.apache.spark.ml.recommendation.ALS.Rating - - @Since("1.4.0") - def this() = this(Identifiable.randomUID("als")) - - /** @group setParam */ - @Since("1.3.0") - def setRank(value: Int): this.type = set(rank, value) - - /** @group setParam */ - @Since("1.3.0") - def setNumUserBlocks(value: Int): this.type = set(numUserBlocks, value) - - /** @group setParam */ - @Since("1.3.0") - def setNumItemBlocks(value: Int): this.type = set(numItemBlocks, value) - - /** @group setParam */ - @Since("1.3.0") - def setImplicitPrefs(value: Boolean): this.type = set(implicitPrefs, value) - - /** @group setParam */ - @Since("1.3.0") - def setAlpha(value: Double): this.type = set(alpha, value) - - /** @group setParam */ - @Since("1.3.0") - def setUserCol(value: String): this.type = set(userCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setItemCol(value: String): this.type = set(itemCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setRatingCol(value: String): this.type = set(ratingCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - - /** @group setParam */ - @Since("1.3.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** @group setParam */ - @Since("1.3.0") - def setRegParam(value: Double): this.type = set(regParam, value) - - /** @group setParam */ - @Since("1.3.0") - def setNonnegative(value: Boolean): this.type = set(nonnegative, value) - - /** @group setParam */ - @Since("1.4.0") - def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) - - /** @group setParam */ - @Since("1.3.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** @group expertSetParam */ - @Since("2.0.0") - def setIntermediateStorageLevel(value: String): this.type = set(intermediateStorageLevel, value) - - /** @group expertSetParam */ - @Since("2.0.0") - def setFinalStorageLevel(value: String): this.type = set(finalStorageLevel, value) - - /** @group expertSetParam */ - @Since("2.2.0") - def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - - /** - * Sets both numUserBlocks and numItemBlocks to the specific value. - * - * @group setParam - */ - @Since("1.3.0") - def setNumBlocks(value: Int): this.type = { - setNumUserBlocks(value) - setNumItemBlocks(value) - this - } - - @Since("2.0.0") - override def fit(dataset: Dataset[_]): ALSModel = instrumented { instr => - transformSchema(dataset.schema) - import dataset.sparkSession.implicits._ - - val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) - val ratings = dataset - .select(checkedCast(col($(userCol))), checkedCast(col($(itemCol))), r) - .rdd - .map { row => - Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) - } - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, - itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, - seed, intermediateStorageLevel, finalStorageLevel, blockSize) - - val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), - numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), - maxIter = $(maxIter), regParam = $(regParam), implicitPrefs = $(implicitPrefs), - alpha = $(alpha), nonnegative = $(nonnegative), - intermediateRDDStorageLevel = StorageLevel.fromString($(intermediateStorageLevel)), - finalRDDStorageLevel = StorageLevel.fromString($(finalStorageLevel)), - checkpointInterval = $(checkpointInterval), seed = $(seed)) - val userDF = userFactors.toDF("id", "features") - val itemDF = itemFactors.toDF("id", "features") - val model = new ALSModel(uid, $(rank), userDF, itemDF).setBlockSize($(blockSize)) - .setParent(this) - copyValues(model) - } - - @Since("1.3.0") - override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) - } - - @Since("1.5.0") - override def copy(extra: ParamMap): ALS = defaultCopy(extra) -} - - -/** - * An implementation of ALS that supports generic ID types, specialized for Int and Long. This is - * exposed as a developer API for users who do need other ID types. But it is not recommended - * because it increases the shuffle size and memory requirement during training. For simplicity, - * users and items must have the same type. The number of distinct users/items should be smaller - * than 2 billion. - */ -object ALS extends DefaultParamsReadable[ALS] with Logging { - - /** - * Rating class for better code readability. - */ - case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) - - @Since("1.6.0") - override def load(path: String): ALS = super.load(path) - - /** Trait for least squares solvers applied to the normal equation. */ - private[recommendation] trait LeastSquaresNESolver extends Serializable { - /** Solves a least squares problem with regularization (possibly with other constraints). */ - def solve(ne: NormalEquation, lambda: Double): Array[Float] - } - - /** Cholesky solver for least square problems. */ - private[recommendation] class CholeskySolver extends LeastSquaresNESolver { - - /** - * Solves a least squares problem with L2 regularization: - * - * min norm(A x - b)^2^ + lambda * norm(x)^2^ - * - * @param ne a [[NormalEquation]] instance that contains AtA, Atb, and n (number of instances) - * @param lambda regularization constant - * @return the solution x - */ - override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { - val k = ne.k - // Add scaled lambda to the diagonals of AtA. - var i = 0 - var j = 2 - while (i < ne.triK) { - ne.ata(i) += lambda - i += j - j += 1 - } - CholeskyDecomposition.solve(ne.ata, ne.atb) - val x = new Array[Float](k) - i = 0 - while (i < k) { - x(i) = ne.atb(i).toFloat - i += 1 - } - ne.reset() - x - } - } - - /** NNLS solver. */ - private[recommendation] class NNLSSolver extends LeastSquaresNESolver { - private var rank: Int = -1 - private var workspace: NNLS.Workspace = _ - private var ata: Array[Double] = _ - private var initialized: Boolean = false - - private def initialize(rank: Int): Unit = { - if (!initialized) { - this.rank = rank - workspace = NNLS.createWorkspace(rank) - ata = new Array[Double](rank * rank) - initialized = true - } else { - require(this.rank == rank) - } - } - - /** - * Solves a nonnegative least squares problem with L2 regularization: - * - * min_x_ norm(A x - b)^2^ + lambda * n * norm(x)^2^ - * subject to x >= 0 - */ - override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { - val rank = ne.k - initialize(rank) - fillAtA(ne.ata, lambda) - val x = NNLS.solve(ata, ne.atb, workspace) - ne.reset() - x.map(x => x.toFloat) - } - - /** - * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square - * matrix that it represents, storing it into destMatrix. - */ - private def fillAtA(triAtA: Array[Double], lambda: Double): Unit = { - var i = 0 - var pos = 0 - var a = 0.0 - while (i < rank) { - var j = 0 - while (j <= i) { - a = triAtA(pos) - ata(i * rank + j) = a - ata(j * rank + i) = a - pos += 1 - j += 1 - } - ata(i * rank + i) += lambda - i += 1 - } - } - } - - /** - * Representing a normal equation to solve the following weighted least squares problem: - * - * minimize \sum,,i,, c,,i,, (a,,i,,^T^ x - d,,i,,)^2^ + lambda * x^T^ x. - * - * Its normal equation is given by - * - * \sum,,i,, c,,i,, (a,,i,, a,,i,,^T^ x - d,,i,, a,,i,,) + lambda * x = 0. - * - * Distributing and letting b,,i,, = c,,i,, * d,,i,, - * - * \sum,,i,, c,,i,, a,,i,, a,,i,,^T^ x - b,,i,, a,,i,, + lambda * x = 0. - */ - private[recommendation] class NormalEquation(val k: Int) extends Serializable { - - /** Number of entries in the upper triangular part of a k-by-k matrix. */ - val triK = k * (k + 1) / 2 - /** A^T^ * A */ - val ata = new Array[Double](triK) - /** A^T^ * b */ - val atb = new Array[Double](k) - - private val da = new Array[Double](k) - private val upper = "U" - - private def copyToDouble(a: Array[Float]): Unit = { - var i = 0 - while (i < k) { - da(i) = a(i) - i += 1 - } - } - - /** Adds an observation. */ - def add(a: Array[Float], b: Double, c: Double = 1.0): NormalEquation = { - require(c >= 0.0) - require(a.length == k) - copyToDouble(a) - blas.dspr(upper, k, c, da, 1, ata) - if (b != 0.0) { - blas.daxpy(k, b, da, 1, atb, 1) - } - this - } - - /** Merges another normal equation object. */ - def merge(other: NormalEquation): NormalEquation = { - require(other.k == k) - blas.daxpy(ata.length, 1.0, other.ata, 1, ata, 1) - blas.daxpy(atb.length, 1.0, other.atb, 1, atb, 1) - this - } - - /** Resets everything to zero, which should be called after each solve. */ - def reset(): Unit = { - ju.Arrays.fill(ata, 0.0) - ju.Arrays.fill(atb, 0.0) - } - } - - def train[ID: ClassTag]( // scalastyle:ignore - ratings: RDD[Rating[ID]], - rank: Int = 10, - numUserBlocks: Int = 10, - numItemBlocks: Int = 10, - maxIter: Int = 10, - regParam: Double = 0.1, - implicitPrefs: Boolean = false, - alpha: Double = 1.0, - nonnegative: Boolean = false, - intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, - finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, - checkpointInterval: Int = 10, - seed: Long = 0L)( - implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - - val isPlatformSupported = DALImplUtils.checkClusterPlatformCompatibility(ratings.sparkContext) - - val (userIdAndFactors, itemIdAndFactors) = - if (implicitPrefs && DALImplUtils.isOAPEnabled() && isPlatformSupported) { - new ALSDALImpl(ratings, rank, maxIter, regParam, alpha, seed).train() - } else { - trainMLlib(ratings, rank, numUserBlocks, numItemBlocks, maxIter, regParam, implicitPrefs, - alpha, nonnegative, intermediateRDDStorageLevel, finalRDDStorageLevel, - checkpointInterval, seed) - } - - (userIdAndFactors, itemIdAndFactors) - } - - private def trainDAL[ID: ClassTag]( - ratings: RDD[Rating[ID]], - rank: Int, - maxIter: Int, - regParam: Double, - alpha: Double, - seed: Long): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - null - } - - /** - * Implementation of the ALS algorithm. - * - * This implementation of the ALS factorization algorithm partitions the two sets of factors among - * Spark workers so as to reduce network communication by only sending one copy of each factor - * vector to each Spark worker on each iteration, and only if needed. This is achieved by - * precomputing some information about the ratings matrix to determine which users require which - * item factors and vice versa. See the Scaladoc for `InBlock` for a detailed explanation of how - * the precomputation is done. - * - * In addition, since each iteration of calculating the factor matrices depends on the known - * ratings, which are spread across Spark partitions, a naive implementation would incur - * significant network communication overhead between Spark workers, as the ratings RDD would be - * repeatedly shuffled during each iteration. This implementation reduces that overhead by - * performing the shuffling operation up front, precomputing each partition's ratings dependencies - * and duplicating those values to the appropriate workers before starting iterations to solve for - * the factor matrices. See the Scaladoc for `OutBlock` for a detailed explanation of how the - * precomputation is done. - * - * Note that the term "rating block" is a bit of a misnomer, as the ratings are not partitioned by - * contiguous blocks from the ratings matrix but by a hash function on the rating's location in - * the matrix. If it helps you to visualize the partitions, it is easier to think of the term - * "block" as referring to a subset of an RDD containing the ratings rather than a contiguous - * submatrix of the ratings matrix. - */ - private def trainMLlib[ID: ClassTag]( // scalastyle:ignore - ratings: RDD[Rating[ID]], - rank: Int = 10, - numUserBlocks: Int = 10, - numItemBlocks: Int = 10, - maxIter: Int = 10, - regParam: Double = 0.1, - implicitPrefs: Boolean = false, - alpha: Double = 1.0, - nonnegative: Boolean = false, - intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, - finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, - checkpointInterval: Int = 10, - seed: Long = 0L)( - implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { - - require(!ratings.isEmpty(), s"No ratings available from $ratings") - require(intermediateRDDStorageLevel != StorageLevel.NONE, - "ALS is not designed to run without persisting intermediate RDDs.") - - val sc = ratings.sparkContext - - // Precompute the rating dependencies of each partition - val userPart = new ALSPartitioner(numUserBlocks) - val itemPart = new ALSPartitioner(numItemBlocks) - val blockRatings = partitionRatings(ratings, userPart, itemPart) - .persist(intermediateRDDStorageLevel) - val (userInBlocks, userOutBlocks) = - makeBlocks("user", blockRatings, userPart, itemPart, intermediateRDDStorageLevel) - userOutBlocks.count() // materialize blockRatings and user blocks - val swappedBlockRatings = blockRatings.map { - case ((userBlockId, itemBlockId), RatingBlock(userIds, itemIds, localRatings)) => - ((itemBlockId, userBlockId), RatingBlock(itemIds, userIds, localRatings)) - } - val (itemInBlocks, itemOutBlocks) = - makeBlocks("item", swappedBlockRatings, itemPart, userPart, intermediateRDDStorageLevel) - itemOutBlocks.count() // materialize item blocks - - // Encoders for storing each user/item's partition ID and index within its partition using a - // single integer; used as an optimization - val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) - val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) - - // These are the user and item factor matrices that, once trained, are multiplied together to - // estimate the rating matrix. The two matrices are stored in RDDs, partitioned by column such - // that each factor column resides on the same Spark worker as its corresponding user or item. - val seedGen = new XORShiftRandom(seed) - var userFactors = initialize(userInBlocks, rank, seedGen.nextLong()) - var itemFactors = initialize(itemInBlocks, rank, seedGen.nextLong()) - - val solver = if (nonnegative) new NNLSSolver else new CholeskySolver - - var previousCheckpointFile: Option[String] = None - val shouldCheckpoint: Int => Boolean = (iter) => - sc.checkpointDir.isDefined && checkpointInterval != -1 && (iter % checkpointInterval == 0) - val deletePreviousCheckpointFile: () => Unit = () => - previousCheckpointFile.foreach { file => - try { - val checkpointFile = new Path(file) - checkpointFile.getFileSystem(sc.hadoopConfiguration).delete(checkpointFile, true) - } catch { - case e: IOException => - logWarning(s"Cannot delete checkpoint file $file:", e) - } - } - - if (implicitPrefs) { - for (iter <- 1 to maxIter) { - userFactors.setName(s"userFactors-$iter").persist(intermediateRDDStorageLevel) - val previousItemFactors = itemFactors - itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder, implicitPrefs, alpha, solver) - previousItemFactors.unpersist() - itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) - // TODO: Generalize PeriodicGraphCheckpointer and use it here. - val deps = itemFactors.dependencies - if (shouldCheckpoint(iter)) { - itemFactors.checkpoint() // itemFactors gets materialized in computeFactors - } - val previousUserFactors = userFactors - userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder, implicitPrefs, alpha, solver) - if (shouldCheckpoint(iter)) { - ALS.cleanShuffleDependencies(sc, deps) - deletePreviousCheckpointFile() - previousCheckpointFile = itemFactors.getCheckpointFile - } - previousUserFactors.unpersist() - } - } else { - var previousCachedItemFactors: Option[RDD[(Int, FactorBlock)]] = None - for (iter <- 0 until maxIter) { - itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder, solver = solver) - if (shouldCheckpoint(iter)) { - itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) - val deps = itemFactors.dependencies - itemFactors.checkpoint() - itemFactors.count() // checkpoint item factors and cut lineage - ALS.cleanShuffleDependencies(sc, deps) - deletePreviousCheckpointFile() - - previousCachedItemFactors.foreach(_.unpersist()) - previousCheckpointFile = itemFactors.getCheckpointFile - previousCachedItemFactors = Option(itemFactors) - } - userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder, solver = solver) - } - } - val userIdAndFactors = userInBlocks - .mapValues(_.srcIds) - .join(userFactors) - .mapPartitions({ items => - items.flatMap { case (_, (ids, factors)) => - ids.view.zip(factors) - } - // Preserve the partitioning because IDs are consistent with the partitioners in userInBlocks - // and userFactors. - }, preservesPartitioning = true) - .setName("userFactors") - .persist(finalRDDStorageLevel) - val itemIdAndFactors = itemInBlocks - .mapValues(_.srcIds) - .join(itemFactors) - .mapPartitions({ items => - items.flatMap { case (_, (ids, factors)) => - ids.view.zip(factors) - } - }, preservesPartitioning = true) - .setName("itemFactors") - .persist(finalRDDStorageLevel) - if (finalRDDStorageLevel != StorageLevel.NONE) { - userIdAndFactors.count() - userInBlocks.unpersist() - userOutBlocks.unpersist() - itemOutBlocks.unpersist() - blockRatings.unpersist() - itemIdAndFactors.count() - itemFactors.unpersist() - itemInBlocks.unpersist() - } - (userIdAndFactors, itemIdAndFactors) - } - - /** - * Factor block that stores factors (Array[Float]) in an Array. - */ - private type FactorBlock = Array[Array[Float]] - - /** - * A mapping of the columns of the items factor matrix that are needed when calculating each row - * of the users factor matrix, and vice versa. - * - * Specifically, when calculating a user factor vector, since only those columns of the items - * factor matrix that correspond to the items that that user has rated are needed, we can avoid - * having to repeatedly copy the entire items factor matrix to each worker later in the algorithm - * by precomputing these dependencies for all users, storing them in an RDD of `OutBlock`s. The - * items' dependencies on the columns of the users factor matrix is computed similarly. - * - * =Example= - * - * Using the example provided in the `InBlock` Scaladoc, `userOutBlocks` would look like the - * following: - * - * {{{ - * userOutBlocks.collect() == Seq( - * 0 -> Array(Array(0, 1), Array(0, 1)), - * 1 -> Array(Array(0), Array(0)) - * ) - * }}} - * - * Each value in this map-like sequence is of type `Array[Array[Int]]`. The values in the - * inner array are the ranks of the sorted user IDs in that partition; so in the example above, - * `Array(0, 1)` in partition 0 refers to user IDs 0 and 6, since when all unique user IDs in - * partition 0 are sorted, 0 is the first ID and 6 is the second. The position of each inner - * array in its enclosing outer array denotes the partition number to which item IDs map; in the - * example, the first `Array(0, 1)` is in position 0 of its outer array, denoting item IDs that - * map to partition 0. - * - * In summary, the data structure encodes the following information: - * - * * There are ratings with user IDs 0 and 6 (encoded in `Array(0, 1)`, where 0 and 1 are the - * indices of the user IDs 0 and 6 on partition 0) whose item IDs map to partitions 0 and 1 - * (represented by the fact that `Array(0, 1)` appears in both the 0th and 1st positions). - * - * * There are ratings with user ID 3 (encoded in `Array(0)`, where 0 is the index of the user - * ID 3 on partition 1) whose item IDs map to partitions 0 and 1 (represented by the fact that - * `Array(0)` appears in both the 0th and 1st positions). - */ - private type OutBlock = Array[Array[Int]] - - /** - * In-link block for computing user and item factor matrices. - * - * The ALS algorithm partitions the columns of the users factor matrix evenly among Spark workers. - * Since each column of the factor matrix is calculated using the known ratings of the correspond- - * ing user, and since the ratings don't change across iterations, the ALS algorithm preshuffles - * the ratings to the appropriate partitions, storing them in `InBlock` objects. - * - * The ratings shuffled by item ID are computed similarly and also stored in `InBlock` objects. - * Note that this means every rating is stored twice, once as shuffled by user ID and once by item - * ID. This is a necessary tradeoff, since in general a rating will not be on the same worker - * when partitioned by user as by item. - * - * =Example= - * - * Say we have a small collection of eight items to offer the seven users in our application. We - * have some known ratings given by the users, as seen in the matrix below: - * - * {{{ - * Items - * 0 1 2 3 4 5 6 7 - * +---+---+---+---+---+---+---+---+ - * 0 | |0.1| | |0.4| | |0.7| - * +---+---+---+---+---+---+---+---+ - * 1 | | | | | | | | | - * +---+---+---+---+---+---+---+---+ - * U 2 | | | | | | | | | - * s +---+---+---+---+---+---+---+---+ - * e 3 | |3.1| | |3.4| | |3.7| - * r +---+---+---+---+---+---+---+---+ - * s 4 | | | | | | | | | - * +---+---+---+---+---+---+---+---+ - * 5 | | | | | | | | | - * +---+---+---+---+---+---+---+---+ - * 6 | |6.1| | |6.4| | |6.7| - * +---+---+---+---+---+---+---+---+ - * }}} - * - * The ratings are represented as an RDD, passed to the `partitionRatings` method as the `ratings` - * parameter: - * - * {{{ - * ratings.collect() == Seq( - * Rating(0, 1, 0.1f), - * Rating(0, 4, 0.4f), - * Rating(0, 7, 0.7f), - * Rating(3, 1, 3.1f), - * Rating(3, 4, 3.4f), - * Rating(3, 7, 3.7f), - * Rating(6, 1, 6.1f), - * Rating(6, 4, 6.4f), - * Rating(6, 7, 6.7f) - * ) - * }}} - * - * Say that we are using two partitions to calculate each factor matrix: - * - * {{{ - * val userPart = new ALSPartitioner(2) - * val itemPart = new ALSPartitioner(2) - * val blockRatings = partitionRatings(ratings, userPart, itemPart) - * }}} - * - * Ratings are mapped to partitions using the user/item IDs modulo the number of partitions. With - * two partitions, ratings with even-valued user IDs are shuffled to partition 0 while those with - * odd-valued user IDs are shuffled to partition 1: - * - * {{{ - * userInBlocks.collect() == Seq( - * 0 -> Seq( - * // Internally, the class stores the ratings in a more optimized format than - * // a sequence of `Rating`s, but for clarity we show it as such here. - * Rating(0, 1, 0.1f), - * Rating(0, 4, 0.4f), - * Rating(0, 7, 0.7f), - * Rating(6, 1, 6.1f), - * Rating(6, 4, 6.4f), - * Rating(6, 7, 6.7f) - * ), - * 1 -> Seq( - * Rating(3, 1, 3.1f), - * Rating(3, 4, 3.4f), - * Rating(3, 7, 3.7f) - * ) - * ) - * }}} - * - * Similarly, ratings with even-valued item IDs are shuffled to partition 0 while those with - * odd-valued item IDs are shuffled to partition 1: - * - * {{{ - * itemInBlocks.collect() == Seq( - * 0 -> Seq( - * Rating(0, 4, 0.4f), - * Rating(3, 4, 3.4f), - * Rating(6, 4, 6.4f) - * ), - * 1 -> Seq( - * Rating(0, 1, 0.1f), - * Rating(0, 7, 0.7f), - * Rating(3, 1, 3.1f), - * Rating(3, 7, 3.7f), - * Rating(6, 1, 6.1f), - * Rating(6, 7, 6.7f) - * ) - * ) - * }}} - * - * @param srcIds src ids (ordered) - * @param dstPtrs dst pointers. Elements in range [dstPtrs(i), dstPtrs(i+1)) of dst indices and - * ratings are associated with srcIds(i). - * @param dstEncodedIndices encoded dst indices - * @param ratings ratings - * @see [[LocalIndexEncoder]] - */ - private[recommendation] case class InBlock[@specialized(Int, Long) ID: ClassTag]( - srcIds: Array[ID], - dstPtrs: Array[Int], - dstEncodedIndices: Array[Int], - ratings: Array[Float]) { - /** Size of the block. */ - def size: Int = ratings.length - require(dstEncodedIndices.length == size) - require(dstPtrs.length == srcIds.length + 1) - } - - /** - * Initializes factors randomly given the in-link blocks. - * - * @param inBlocks in-link blocks - * @param rank rank - * @return initialized factor blocks - */ - private def initialize[ID]( - inBlocks: RDD[(Int, InBlock[ID])], - rank: Int, - seed: Long): RDD[(Int, FactorBlock)] = { - // Choose a unit vector uniformly at random from the unit sphere, but from the - // "first quadrant" where all elements are nonnegative. This can be done by choosing - // elements distributed as Normal(0,1) and taking the absolute value, and then normalizing. - // This appears to create factorizations that have a slightly better reconstruction - // (<1%) compared picking elements uniformly at random in [0,1]. - inBlocks.mapPartitions({ iter => - iter.map { - case (srcBlockId, inBlock) => - val random = new XORShiftRandom(byteswap64(seed ^ srcBlockId)) - val factors = Array.fill(inBlock.srcIds.length) { - val factor = Array.fill(rank)(random.nextGaussian().toFloat) - val nrm = blas.snrm2(rank, factor, 1) - blas.sscal(rank, 1.0f / nrm, factor, 1) - factor - } - (srcBlockId, factors) - } - }, preservesPartitioning = true) - } - - /** - * A rating block that contains src IDs, dst IDs, and ratings, stored in primitive arrays. - */ - private[recommendation] case class RatingBlock[@specialized(Int, Long) ID: ClassTag]( - srcIds: Array[ID], - dstIds: Array[ID], - ratings: Array[Float]) { - /** Size of the block. */ - def size: Int = srcIds.length - require(dstIds.length == srcIds.length) - require(ratings.length == srcIds.length) - } - - /** - * Builder for [[RatingBlock]]. `mutable.ArrayBuilder` is used to avoid boxing/unboxing. - */ - private[recommendation] class RatingBlockBuilder[@specialized(Int, Long) ID: ClassTag] - extends Serializable { - - private val srcIds = mutable.ArrayBuilder.make[ID] - private val dstIds = mutable.ArrayBuilder.make[ID] - private val ratings = mutable.ArrayBuilder.make[Float] - var size = 0 - - /** Adds a rating. */ - def add(r: Rating[ID]): this.type = { - size += 1 - srcIds += r.user - dstIds += r.item - ratings += r.rating - this - } - - /** Merges another [[RatingBlockBuilder]]. */ - def merge(other: RatingBlock[ID]): this.type = { - size += other.srcIds.length - srcIds ++= other.srcIds - dstIds ++= other.dstIds - ratings ++= other.ratings - this - } - - /** Builds a [[RatingBlock]]. */ - def build(): RatingBlock[ID] = { - RatingBlock[ID](srcIds.result(), dstIds.result(), ratings.result()) - } - } - - /** - * Groups an RDD of [[Rating]]s by the user partition and item partition to which each `Rating` - * maps according to the given partitioners. The returned pair RDD holds the ratings, encoded in - * a memory-efficient format but otherwise unchanged, keyed by the (user partition ID, item - * partition ID) pair. - * - * Performance note: This is an expensive operation that performs an RDD shuffle. - * - * Implementation note: This implementation produces the same result as the following but - * generates fewer intermediate objects: - * - * {{{ - * ratings.map { r => - * ((srcPart.getPartition(r.user), dstPart.getPartition(r.item)), r) - * }.aggregateByKey(new RatingBlockBuilder)( - * seqOp = (b, r) => b.add(r), - * combOp = (b0, b1) => b0.merge(b1.build())) - * .mapValues(_.build()) - * }}} - * - * @param ratings raw ratings - * @param srcPart partitioner for src IDs - * @param dstPart partitioner for dst IDs - * @return an RDD of rating blocks in the form of ((srcBlockId, dstBlockId), ratingBlock) - */ - private def partitionRatings[ID: ClassTag]( - ratings: RDD[Rating[ID]], - srcPart: Partitioner, - dstPart: Partitioner): RDD[((Int, Int), RatingBlock[ID])] = { - val numPartitions = srcPart.numPartitions * dstPart.numPartitions - ratings.mapPartitions { iter => - val builders = Array.fill(numPartitions)(new RatingBlockBuilder[ID]) - iter.flatMap { r => - val srcBlockId = srcPart.getPartition(r.user) - val dstBlockId = dstPart.getPartition(r.item) - val idx = srcBlockId + srcPart.numPartitions * dstBlockId - val builder = builders(idx) - builder.add(r) - if (builder.size >= 2048) { // 2048 * (3 * 4) = 24k - builders(idx) = new RatingBlockBuilder - Iterator.single(((srcBlockId, dstBlockId), builder.build())) - } else { - Iterator.empty - } - } ++ { - builders.view.zipWithIndex.filter(_._1.size > 0).map { case (block, idx) => - val srcBlockId = idx % srcPart.numPartitions - val dstBlockId = idx / srcPart.numPartitions - ((srcBlockId, dstBlockId), block.build()) - } - } - }.groupByKey().mapValues { blocks => - val builder = new RatingBlockBuilder[ID] - blocks.foreach(builder.merge) - builder.build() - }.setName("ratingBlocks") - } - - /** - * Builder for uncompressed in-blocks of (srcId, dstEncodedIndex, rating) tuples. - * - * @param encoder encoder for dst indices - */ - private[recommendation] class UncompressedInBlockBuilder[@specialized(Int, Long) ID: ClassTag]( - encoder: LocalIndexEncoder)( - implicit ord: Ordering[ID]) { - - private val srcIds = mutable.ArrayBuilder.make[ID] - private val dstEncodedIndices = mutable.ArrayBuilder.make[Int] - private val ratings = mutable.ArrayBuilder.make[Float] - - /** - * Adds a dst block of (srcId, dstLocalIndex, rating) tuples. - * - * @param dstBlockId dst block ID - * @param srcIds original src IDs - * @param dstLocalIndices dst local indices - * @param ratings ratings - */ - def add( - dstBlockId: Int, - srcIds: Array[ID], - dstLocalIndices: Array[Int], - ratings: Array[Float]): this.type = { - val sz = srcIds.length - require(dstLocalIndices.length == sz) - require(ratings.length == sz) - this.srcIds ++= srcIds - this.ratings ++= ratings - var j = 0 - while (j < sz) { - this.dstEncodedIndices += encoder.encode(dstBlockId, dstLocalIndices(j)) - j += 1 - } - this - } - - /** Builds a [[UncompressedInBlock]]. */ - def build(): UncompressedInBlock[ID] = { - new UncompressedInBlock(srcIds.result(), dstEncodedIndices.result(), ratings.result()) - } - } - - /** - * A block of (srcId, dstEncodedIndex, rating) tuples stored in primitive arrays. - */ - private[recommendation] class UncompressedInBlock[@specialized(Int, Long) ID: ClassTag]( - val srcIds: Array[ID], - val dstEncodedIndices: Array[Int], - val ratings: Array[Float])( - implicit ord: Ordering[ID]) { - - /** Size the of block. */ - def length: Int = srcIds.length - - /** - * Compresses the block into an `InBlock`. The algorithm is the same as converting a sparse - * matrix from coordinate list (COO) format into compressed sparse column (CSC) format. - * Sorting is done using Spark's built-in Timsort to avoid generating too many objects. - */ - def compress(): InBlock[ID] = { - val sz = length - assert(sz > 0, "Empty in-link block should not exist.") - sort() - val uniqueSrcIdsBuilder = mutable.ArrayBuilder.make[ID] - val dstCountsBuilder = mutable.ArrayBuilder.make[Int] - var preSrcId = srcIds(0) - uniqueSrcIdsBuilder += preSrcId - var curCount = 1 - var i = 1 - while (i < sz) { - val srcId = srcIds(i) - if (srcId != preSrcId) { - uniqueSrcIdsBuilder += srcId - dstCountsBuilder += curCount - preSrcId = srcId - curCount = 0 - } - curCount += 1 - i += 1 - } - dstCountsBuilder += curCount - val uniqueSrcIds = uniqueSrcIdsBuilder.result() - val numUniqueSrdIds = uniqueSrcIds.length - val dstCounts = dstCountsBuilder.result() - val dstPtrs = new Array[Int](numUniqueSrdIds + 1) - var sum = 0 - i = 0 - while (i < numUniqueSrdIds) { - sum += dstCounts(i) - i += 1 - dstPtrs(i) = sum - } - InBlock(uniqueSrcIds, dstPtrs, dstEncodedIndices, ratings) - } - - private def sort(): Unit = { - val sz = length - // Since there might be interleaved log messages, we insert a unique id for easy pairing. - val sortId = Utils.random.nextInt() - logDebug(s"Start sorting an uncompressed in-block of size $sz. (sortId = $sortId)") - val start = System.nanoTime() - val sorter = new Sorter(new UncompressedInBlockSort[ID]) - sorter.sort(this, 0, length, Ordering[KeyWrapper[ID]]) - val duration = (System.nanoTime() - start) / 1e9 - logDebug(s"Sorting took $duration seconds. (sortId = $sortId)") - } - } - - /** - * A wrapper that holds a primitive key. - * - * @see [[UncompressedInBlockSort]] - */ - private class KeyWrapper[@specialized(Int, Long) ID: ClassTag]( - implicit ord: Ordering[ID]) extends Ordered[KeyWrapper[ID]] { - - var key: ID = _ - - override def compare(that: KeyWrapper[ID]): Int = { - ord.compare(key, that.key) - } - - def setKey(key: ID): this.type = { - this.key = key - this - } - } - - /** - * [[SortDataFormat]] of [[UncompressedInBlock]] used by [[Sorter]]. - */ - private class UncompressedInBlockSort[@specialized(Int, Long) ID: ClassTag]( - implicit ord: Ordering[ID]) - extends SortDataFormat[KeyWrapper[ID], UncompressedInBlock[ID]] { - - override def newKey(): KeyWrapper[ID] = new KeyWrapper() - - override def getKey( - data: UncompressedInBlock[ID], - pos: Int, - reuse: KeyWrapper[ID]): KeyWrapper[ID] = { - if (reuse == null) { - new KeyWrapper().setKey(data.srcIds(pos)) - } else { - reuse.setKey(data.srcIds(pos)) - } - } - - override def getKey( - data: UncompressedInBlock[ID], - pos: Int): KeyWrapper[ID] = { - getKey(data, pos, null) - } - - private def swapElements[@specialized(Int, Float) T]( - data: Array[T], - pos0: Int, - pos1: Int): Unit = { - val tmp = data(pos0) - data(pos0) = data(pos1) - data(pos1) = tmp - } - - override def swap(data: UncompressedInBlock[ID], pos0: Int, pos1: Int): Unit = { - swapElements(data.srcIds, pos0, pos1) - swapElements(data.dstEncodedIndices, pos0, pos1) - swapElements(data.ratings, pos0, pos1) - } - - override def copyRange( - src: UncompressedInBlock[ID], - srcPos: Int, - dst: UncompressedInBlock[ID], - dstPos: Int, - length: Int): Unit = { - System.arraycopy(src.srcIds, srcPos, dst.srcIds, dstPos, length) - System.arraycopy(src.dstEncodedIndices, srcPos, dst.dstEncodedIndices, dstPos, length) - System.arraycopy(src.ratings, srcPos, dst.ratings, dstPos, length) - } - - override def allocate(length: Int): UncompressedInBlock[ID] = { - new UncompressedInBlock( - new Array[ID](length), new Array[Int](length), new Array[Float](length)) - } - - override def copyElement( - src: UncompressedInBlock[ID], - srcPos: Int, - dst: UncompressedInBlock[ID], - dstPos: Int): Unit = { - dst.srcIds(dstPos) = src.srcIds(srcPos) - dst.dstEncodedIndices(dstPos) = src.dstEncodedIndices(srcPos) - dst.ratings(dstPos) = src.ratings(srcPos) - } - } - - /** - * Creates in-blocks and out-blocks from rating blocks. - * - * @param prefix prefix for in/out-block names - * @param ratingBlocks rating blocks - * @param srcPart partitioner for src IDs - * @param dstPart partitioner for dst IDs - * @return (in-blocks, out-blocks) - */ - private def makeBlocks[ID: ClassTag]( - prefix: String, - ratingBlocks: RDD[((Int, Int), RatingBlock[ID])], - srcPart: Partitioner, - dstPart: Partitioner, - storageLevel: StorageLevel)( - implicit srcOrd: Ordering[ID]): (RDD[(Int, InBlock[ID])], RDD[(Int, OutBlock)]) = { - val inBlocks = ratingBlocks.map { - case ((srcBlockId, dstBlockId), RatingBlock(srcIds, dstIds, ratings)) => - // The implementation is a faster version of - // val dstIdToLocalIndex = dstIds.toSet.toSeq.sorted.zipWithIndex.toMap - val start = System.nanoTime() - val dstIdSet = new OpenHashSet[ID](1 << 20) - dstIds.foreach(dstIdSet.add) - val sortedDstIds = new Array[ID](dstIdSet.size) - var i = 0 - var pos = dstIdSet.nextPos(0) - while (pos != -1) { - sortedDstIds(i) = dstIdSet.getValue(pos) - pos = dstIdSet.nextPos(pos + 1) - i += 1 - } - assert(i == dstIdSet.size) - Sorting.quickSort(sortedDstIds) - val dstIdToLocalIndex = new OpenHashMap[ID, Int](sortedDstIds.length) - i = 0 - while (i < sortedDstIds.length) { - dstIdToLocalIndex.update(sortedDstIds(i), i) - i += 1 - } - logDebug( - "Converting to local indices took " + (System.nanoTime() - start) / 1e9 + " seconds.") - val dstLocalIndices = dstIds.map(dstIdToLocalIndex.apply) - (srcBlockId, (dstBlockId, srcIds, dstLocalIndices, ratings)) - }.groupByKey(new ALSPartitioner(srcPart.numPartitions)) - .mapValues { iter => - val builder = - new UncompressedInBlockBuilder[ID](new LocalIndexEncoder(dstPart.numPartitions)) - iter.foreach { case (dstBlockId, srcIds, dstLocalIndices, ratings) => - builder.add(dstBlockId, srcIds, dstLocalIndices, ratings) - } - builder.build().compress() - }.setName(prefix + "InBlocks") - .persist(storageLevel) - val outBlocks = inBlocks.mapValues { case InBlock(srcIds, dstPtrs, dstEncodedIndices, _) => - val encoder = new LocalIndexEncoder(dstPart.numPartitions) - val activeIds = Array.fill(dstPart.numPartitions)(mutable.ArrayBuilder.make[Int]) - var i = 0 - val seen = new Array[Boolean](dstPart.numPartitions) - while (i < srcIds.length) { - var j = dstPtrs(i) - ju.Arrays.fill(seen, false) - while (j < dstPtrs(i + 1)) { - val dstBlockId = encoder.blockId(dstEncodedIndices(j)) - if (!seen(dstBlockId)) { - activeIds(dstBlockId) += i // add the local index in this out-block - seen(dstBlockId) = true - } - j += 1 - } - i += 1 - } - activeIds.map { x => - x.result() - } - }.setName(prefix + "OutBlocks") - .persist(storageLevel) - (inBlocks, outBlocks) - } - - /** - * Compute dst factors by constructing and solving least square problems. - * - * @param srcFactorBlocks src factors - * @param srcOutBlocks src out-blocks - * @param dstInBlocks dst in-blocks - * @param rank rank - * @param regParam regularization constant - * @param srcEncoder encoder for src local indices - * @param implicitPrefs whether to use implicit preference - * @param alpha the alpha constant in the implicit preference formulation - * @param solver solver for least squares problems - * @return dst factors - */ - private def computeFactors[ID]( - srcFactorBlocks: RDD[(Int, FactorBlock)], - srcOutBlocks: RDD[(Int, OutBlock)], - dstInBlocks: RDD[(Int, InBlock[ID])], - rank: Int, - regParam: Double, - srcEncoder: LocalIndexEncoder, - implicitPrefs: Boolean = false, - alpha: Double = 1.0, - solver: LeastSquaresNESolver): RDD[(Int, FactorBlock)] = { - val numSrcBlocks = srcFactorBlocks.partitions.length - val YtY = if (implicitPrefs) Some(computeYtY(srcFactorBlocks, rank)) else None - val srcOut = srcOutBlocks.join(srcFactorBlocks).flatMap { - case (srcBlockId, (srcOutBlock, srcFactors)) => - srcOutBlock.view.zipWithIndex.map { case (activeIndices, dstBlockId) => - (dstBlockId, (srcBlockId, activeIndices.map(idx => srcFactors(idx)))) - } - } - val merged = srcOut.groupByKey(new ALSPartitioner(dstInBlocks.partitions.length)) - - // SPARK-28927: Nondeterministic RDDs causes inconsistent in/out blocks in case of rerun. - // It can cause runtime error when matching in/out user/item blocks. - val isBlockRDDNondeterministic = - dstInBlocks.outputDeterministicLevel == DeterministicLevel.INDETERMINATE || - srcOutBlocks.outputDeterministicLevel == DeterministicLevel.INDETERMINATE - - dstInBlocks.join(merged).mapValues { - case (InBlock(dstIds, srcPtrs, srcEncodedIndices, ratings), srcFactors) => - val sortedSrcFactors = new Array[FactorBlock](numSrcBlocks) - srcFactors.foreach { case (srcBlockId, factors) => - sortedSrcFactors(srcBlockId) = factors - } - val dstFactors = new Array[Array[Float]](dstIds.length) - var j = 0 - val ls = new NormalEquation(rank) - while (j < dstIds.length) { - ls.reset() - if (implicitPrefs) { - ls.merge(YtY.get) - } - var i = srcPtrs(j) - var numExplicits = 0 - while (i < srcPtrs(j + 1)) { - val encoded = srcEncodedIndices(i) - val blockId = srcEncoder.blockId(encoded) - val localIndex = srcEncoder.localIndex(encoded) - var srcFactor: Array[Float] = null - try { - srcFactor = sortedSrcFactors(blockId)(localIndex) - } catch { - case a: ArrayIndexOutOfBoundsException if isBlockRDDNondeterministic => - val errMsg = "A failure detected when matching In/Out blocks of users/items. " + - "Because at least one In/Out block RDD is found to be nondeterministic now, " + - "the issue is probably caused by nondeterministic input data. You can try to " + - "checkpoint training data to make it deterministic. If you do `repartition` + " + - "`sample` or `randomSplit`, you can also try to sort it before `sample` or " + - "`randomSplit` to make it deterministic." - throw new SparkException(errMsg, a) - } - val rating = ratings(i) - if (implicitPrefs) { - // Extension to the original paper to handle rating < 0. confidence is a function - // of |rating| instead so that it is never negative. c1 is confidence - 1. - val c1 = alpha * math.abs(rating) - // For rating <= 0, the corresponding preference is 0. So the second argument of add - // is only there for rating > 0. - if (rating > 0.0) { - numExplicits += 1 - } - ls.add(srcFactor, if (rating > 0.0) 1.0 + c1 else 0.0, c1) - } else { - ls.add(srcFactor, rating) - numExplicits += 1 - } - i += 1 - } - // Weight lambda by the number of explicit ratings based on the ALS-WR paper. - dstFactors(j) = solver.solve(ls, numExplicits * regParam) - j += 1 - } - dstFactors - } - } - - /** - * Computes the Gramian matrix of user or item factors, which is only used in implicit preference. - * Caching of the input factors is handled in [[ALS#train]]. - */ - private def computeYtY(factorBlocks: RDD[(Int, FactorBlock)], rank: Int): NormalEquation = { - factorBlocks.values.aggregate(new NormalEquation(rank))( - seqOp = (ne, factors) => { - factors.foreach(ne.add(_, 0.0)) - ne - }, - combOp = (ne1, ne2) => ne1.merge(ne2)) - } - - /** - * Encoder for storing (blockId, localIndex) into a single integer. - * - * We use the leading bits (including the sign bit) to store the block id and the rest to store - * the local index. This is based on the assumption that users/items are approximately evenly - * partitioned. With this assumption, we should be able to encode two billion distinct values. - * - * @param numBlocks number of blocks - */ - private[recommendation] class LocalIndexEncoder(numBlocks: Int) extends Serializable { - - require(numBlocks > 0, s"numBlocks must be positive but found $numBlocks.") - - private[this] final val numLocalIndexBits = - math.min(java.lang.Integer.numberOfLeadingZeros(numBlocks - 1), 31) - private[this] final val localIndexMask = (1 << numLocalIndexBits) - 1 - - /** Encodes a (blockId, localIndex) into a single integer. */ - def encode(blockId: Int, localIndex: Int): Int = { - require(blockId < numBlocks) - require((localIndex & ~localIndexMask) == 0) - (blockId << numLocalIndexBits) | localIndex - } - - /** Gets the block id from an encoded index. */ - @inline - def blockId(encoded: Int): Int = { - encoded >>> numLocalIndexBits - } - - /** Gets the local index from an encoded index. */ - @inline - def localIndex(encoded: Int): Int = { - encoded & localIndexMask - } - } - - /** - * Partitioner used by ALS. We require that getPartition is a projection. That is, for any key k, - * we have getPartition(getPartition(k)) = getPartition(k). Since the default HashPartitioner - * satisfies this requirement, we simply use a type alias here. - */ - private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner - - /** - * Private function to clean up all of the shuffles files from the dependencies and their parents. - */ - private[spark] def cleanShuffleDependencies[T]( - sc: SparkContext, - deps: Seq[Dependency[_]], - blocking: Boolean = false): Unit = { - // If there is no reference tracking we skip clean up. - sc.cleaner.foreach { cleaner => - /** - * Clean the shuffles & all of its parents. - */ - def cleanEagerly(dep: Dependency[_]): Unit = { - if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { - val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId - cleaner.doCleanupShuffle(shuffleId, blocking) - } - val rdd = dep.rdd - val rddDeps = rdd.dependencies - if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { - rddDeps.foreach(cleanEagerly) - } - } - deps.foreach(cleanEagerly) - } - } -} diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..7fdeec20c --- /dev/null +++ b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import scala.annotation.varargs + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.Row +import org.apache.spark.ml.util._ + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( + data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } +} diff --git a/mllib-dal/src/test/resources/log4j.properties b/mllib-dal/src/test/resources/log4j.properties index ff29121c2..18bd98f9d 100644 --- a/mllib-dal/src/test/resources/log4j.properties +++ b/mllib-dal/src/test/resources/log4j.properties @@ -39,4 +39,4 @@ log4j.logger.parquet=ERROR log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR -log4j.logger.org.apache.spark.ml.util.LibLoader=DEBUG +log4j.logger.com.intel.oap.mllib.LibLoader=DEBUG diff --git a/mllib-dal/src/test/scala/org/apache/spark/ml/feature/MLlibPCASuite.scala b/mllib-dal/src/test/scala/org/apache/spark/ml/feature/MLlibPCASuite.scala index 2d345e48c..b80a941cd 100644 --- a/mllib-dal/src/test/scala/org/apache/spark/ml/feature/MLlibPCASuite.scala +++ b/mllib-dal/src/test/scala/org/apache/spark/ml/feature/MLlibPCASuite.scala @@ -66,7 +66,7 @@ class MLlibPCASuite extends MLTest with DefaultReadWriteTest { val pcaModel = pca.fit(df) - println("Intel MLlib: ") + println("OAP MLlib: ") println(s"Principle Components: \n${pcaModel.pc}") println(s"Explained Variance: \n${pcaModel.explainedVariance}") diff --git a/mllib-dal/test-cpu-gpu.sh b/mllib-dal/test-cpu-gpu.sh deleted file mode 100755 index dc6ddd5ee..000000000 --- a/mllib-dal/test-cpu-gpu.sh +++ /dev/null @@ -1,119 +0,0 @@ -#!/usr/bin/env bash - -# Check envs for building -if [[ -z $JAVA_HOME ]]; then - echo JAVA_HOME not defined! - exit 1 -fi - -if [[ -z $(which mvn) ]]; then - echo Maven not found! - exit 1 -fi - -if [[ -z $DAALROOT ]]; then - echo DAALROOT not defined! - exit 1 -fi - -if [[ -z $TBBROOT ]]; then - echo TBBROOT not defined! - exit 1 -fi - -if [[ -z $CCL_ROOT ]]; then - echo CCL_ROOT not defined! - exit 1 -fi - -versionArray=( - spark-3.0.0 \ - spark-3.0.1 \ - spark-3.0.2 \ - spark-3.1.1 -) - -suiteArray=( - "clustering.MLlibKMeansSuite" \ - "feature.MLlibPCASuite" \ - "recommendation.MLlibALSSuite" \ - "classification.MLlibNaiveBayesSuite" \ - "regression.MLlibLinearRegressionSuite" \ - "stat.MLlibCorrelationSuite" - -) - -# Set default version -SPARK_VER=spark-3.1.1 -MVN_NO_TRANSFER_PROGRESS= - -print_usage() { - echo - echo Usage: ./test.sh [-p spark-x.x.x] [-q] [-h] [test suite name] - echo - echo Supported Spark versions: - for version in ${versionArray[*]} - do - echo " $version" - done - echo - echo Supported Test suites: - for suite in ${suiteArray[*]} - do - echo " $suite" - done - echo -} - -while getopts "hqp:" opt -do -case $opt in - p) SPARK_VER=$OPTARG ;; - q) MVN_NO_TRANSFER_PROGRESS=--no-transfer-progress ;; - h | *) - print_usage - exit 1 - ;; -esac -done - -shift "$((OPTIND-1))" - -print_usage - -export PLATFORM_PROFILE=CPU_GPU_PROFILE - -echo === Testing Environments === -echo JAVA_HOME=$JAVA_HOME -echo DAALROOT=$DAALROOT -echo TBBROOT=$TBBROOT -echo CCL_ROOT=$CCL_ROOT -echo Maven Version: $(mvn -v | head -n 1 | cut -f3 -d" ") -echo Clang Version: $(clang -dumpversion) -echo Spark Version: $SPARK_VER -echo Platform Profile: $PLATFORM_PROFILE -echo ============================ - -SUITE=$1 - -if [[ ! ${versionArray[*]} =~ $SPARK_VER ]]; then - echo Error: $SPARK_VER version is not supported! - exit 1 -fi - -if [[ ! ${suiteArray[*]} =~ $SUITE ]]; then - echo Error: $SUITE test suite is not supported! - exit 1 -fi - -if [[ -z $SUITE ]]; then - echo - echo Testing ALL suites... - echo - mvn $MVN_NO_TRANSFER_PROGRESS -P$SPARK_VER -Dtest=none clean test -else - echo - echo Testing org.apache.spark.ml.$SUITE ... - echo - mvn $MVN_NO_TRANSFER_PROGRESS -P$SPARK_VER -Dtest=none -DwildcardSuites=org.apache.spark.ml.$SUITE clean test -fi diff --git a/mllib-dal/test.sh b/mllib-dal/test.sh index 13235cb26..7147b59b3 100755 --- a/mllib-dal/test.sh +++ b/mllib-dal/test.sh @@ -1,11 +1,23 @@ #!/usr/bin/env bash +# exit when any command fails +set -e + +# keep track of the last executed command +trap 'last_command=$current_command; current_command=$BASH_COMMAND' DEBUG +# echo an error message before exiting +trap 'echo "\"${last_command}\" command filed with exit code $?."' EXIT + +SCRIPT_DIR=$( cd $(dirname ${BASH_SOURCE[0]}) && pwd ) +OAP_MLLIB_ROOT=$(cd $SCRIPT_DIR/.. && pwd) +source $OAP_MLLIB_ROOT/RELEASE + if [[ -n $DAALROOT ]]; then echo echo ==================================================================================== echo WARNING: DAALROOT detected. It is recommended to test without oneAPI environment! echo ==================================================================================== - echo + echo fi # Unset FI_PROVIDER_PATH if present otherwise may hang @@ -16,7 +28,7 @@ if [[ -n $FI_PROVIDER_PATH ]]; then echo ==================================================================================== fi -if [[ ! -f target/oap-mllib-1.2.0.jar ]]; then +if [[ ! -f target/oap-mllib-$OAP_MLLIB_VERSION.jar ]]; then echo Please run ./build.sh first to do a complete build before testing! exit 1 fi @@ -34,13 +46,6 @@ fi export OAP_MLLIB_TESTING=true -versionArray=( - spark-3.0.0 \ - spark-3.0.1 \ - spark-3.0.2 \ - spark-3.1.1 -) - suiteArray=( "clustering.MLlibKMeansSuite" \ "feature.MLlibPCASuite" \ @@ -50,32 +55,22 @@ suiteArray=( "stat.MLlibCorrelationSuite" ) -# Set default version -SPARK_VER=spark-3.1.1 MVN_NO_TRANSFER_PROGRESS= print_usage() { echo - echo Usage: ./test.sh [-p spark-x.x.x] [-q] [-h] [test suite name] - echo - echo Supported Spark versions: - for version in ${versionArray[*]} - do - echo " $version" - done + echo "Usage: ./test.sh [-p ] [-q] [-h] " echo - echo Supported Test suites: - for suite in ${suiteArray[*]} - do - echo " $suite" - done + echo "-p Supported Platform Profiles:" + echo " CPU_ONLY_PROFILE" + echo " CPU_GPU_PROFILE" echo } -while getopts "hqp:" opt +while getopts "p:qh" opt do case $opt in - p) SPARK_VER=$OPTARG ;; + p) PLATFORM_OPT=$OPTARG ;; q) MVN_NO_TRANSFER_PROGRESS=--no-transfer-progress ;; h | *) print_usage @@ -86,21 +81,14 @@ done shift "$((OPTIND-1))" -print_usage - -export PLATFORM_PROFILE=CPU_ONLY_PROFILE - -echo === Testing Environments === -echo JAVA_HOME=$JAVA_HOME -echo Maven Version: $(mvn -v | head -n 1 | cut -f3 -d" ") -echo Spark Version: $SPARK_VER -echo Platform Profile: $PLATFORM_PROFILE -echo ============================ - SUITE=$1 -if [[ ! ${versionArray[*]} =~ $SPARK_VER ]]; then - echo Error: $SPARK_VER version is not supported! +print_usage + +if [[ ! ($PLATFORM_PROFILE == CPU_ONLY_PROFILE || $PLATFORM_PROFILE == CPU_GPU_PROFILE) ]]; then + echo + echo Platform Profile should be CPU_ONLY_PROFILE or CPU_GPU_PROFILE, but \"$PLATFORM_PROFILE\" found! + echo exit 1 fi @@ -109,14 +97,31 @@ if [[ ! ${suiteArray[*]} =~ $SUITE ]]; then exit 1 fi +# Import RELEASE envs +SCRIPT_DIR=$( cd $(dirname ${BASH_SOURCE[0]}) && pwd ) +OAP_MLLIB_ROOT=$(cd $SCRIPT_DIR/.. && pwd) +source $OAP_MLLIB_ROOT/RELEASE + +export SPARK_VERSION=${SPARK_OPT:-$SPARK_VERSION} +export PLATFORM_PROFILE=${PLATFORM_OPT:-$PLATFORM_PROFILE} + +echo +echo === Testing Environments === +echo JAVA_HOME=$JAVA_HOME +echo Maven Version: $(mvn -v | head -n 1 | cut -f3 -d" ") +echo Spark Version: $SPARK_VERSION +echo Platform Profile: $PLATFORM_PROFILE +echo ============================ +echo + if [[ -z $SUITE ]]; then echo echo Testing ALL suites... echo - mvn $MVN_NO_TRANSFER_PROGRESS -P$SPARK_VER -Dtest=none test + mvn $MVN_NO_TRANSFER_PROGRESS -Dspark.version=$SPARK_VERSION -Dtest=none test else echo echo Testing org.apache.spark.ml.$SUITE ... echo - mvn $MVN_NO_TRANSFER_PROGRESS -P$SPARK_VER -Dtest=none -DwildcardSuites=org.apache.spark.ml.$SUITE test + mvn $MVN_NO_TRANSFER_PROGRESS -Dspark.version=$SPARK_VERSION -Dtest=none -DwildcardSuites=org.apache.spark.ml.$SUITE test fi From 9b33201bc8e48af62badaf78dbe18ba2c88b4ef8 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Wed, 24 Nov 2021 21:20:18 +0800 Subject: [PATCH 03/13] 1. add Summarizer example and CI 2. refactor Low-Order Moments 3. fix comments Signed-off-by: minmingzhu --- examples/build-all-scala.sh | 2 +- examples/run-all-scala.sh | 2 +- examples/summarizer/pom.xml | 4 +- .../oap/mllib/stat/SummarizerResult.java | 8 + .../spark/mllib/stat/SummarizerResult.java | 9 - .../src/main/native/SummarizerDALImpl.cpp | 381 +++++++++--------- ...m_intel_oap_mllib_stat_SummarizerDALImpl.h | 21 + ...pache_spark_mllib_stat_SummarizerDALImpl.h | 21 - .../scala/com/intel/oap/mllib/Utils.scala | 6 +- .../oap}/mllib/stat/SummarizerDALImpl.scala | 33 +- .../intel/oap/mllib/stat/SummarizerShim.scala | 43 ++ .../MultivariateStatisticalDALSummary.scala | 0 .../apache/spark/mllib/stat/Statistics.scala | 119 ++++++ .../mllib/stat/spark320/Statistics.scala | 73 ++++ .../apache/spark/mllib/stat/Statistics.scala | 275 ------------- .../MultivariateStatisticalDALSummary.scala | 65 --- .../apache/spark/mllib/stat/Statistics.scala | 275 ------------- .../MultivariateStatisticalDALSummary.scala | 65 --- .../apache/spark/mllib/stat/Statistics.scala | 275 ------------- .../MultivariateStatisticalDALSummary.scala | 65 --- .../apache/spark/mllib/stat/Statistics.scala | 275 ------------- 21 files changed, 478 insertions(+), 1539 deletions(-) create mode 100644 mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java delete mode 100644 mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h delete mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h rename mllib-dal/src/main/scala/{org/apache/spark => com/intel/oap}/mllib/stat/SummarizerDALImpl.scala (70%) create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala rename mllib-dal/src/{spark-3.0.0 => }/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala (100%) create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala delete mode 100644 mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala delete mode 100644 mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala delete mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala delete mode 100644 mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala diff --git a/examples/build-all-scala.sh b/examples/build-all-scala.sh index 883f87868..8babb9d97 100755 --- a/examples/build-all-scala.sh +++ b/examples/build-all-scala.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) +exampleDirs=(kmeans pca als naive-bayes linear-regression correlation summarizer) for dir in ${exampleDirs[*]} do diff --git a/examples/run-all-scala.sh b/examples/run-all-scala.sh index 269341848..04bab7f8a 100755 --- a/examples/run-all-scala.sh +++ b/examples/run-all-scala.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) +exampleDirs=(kmeans pca als naive-bayes linear-regression correlation summarizer) for dir in ${exampleDirs[*]} do diff --git a/examples/summarizer/pom.xml b/examples/summarizer/pom.xml index 2793bc681..51aa36355 100644 --- a/examples/summarizer/pom.xml +++ b/examples/summarizer/pom.xml @@ -4,7 +4,7 @@ com.intel.oap oap-mllib-examples - 1.2.0 + 1.3.0 jar SummaryStatisticsExample @@ -12,7 +12,7 @@ UTF-8 - 1.2.0 + 1.3.0 2.12.10 2.12 3.1.1 diff --git a/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java new file mode 100644 index 000000000..d70499768 --- /dev/null +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java @@ -0,0 +1,8 @@ +package com.intel.oap.mllib.stat; + +public class SummarizerResult { + public long meanNumericTable; + public long varianceNumericTable; + public long minimumNumericTable; + public long maximumNumericTable; +} diff --git a/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java b/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java deleted file mode 100644 index d34485004..000000000 --- a/mllib-dal/src/main/java/org/apache/spark/mllib/stat/SummarizerResult.java +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.spark.mllib.stat; - -public class SummarizerResult { - public long meanNumericTable; - public long varianceNumericTable; - public long minimumNumericTable; - public long maximumNumericTable; - -} \ No newline at end of file diff --git a/mllib-dal/src/main/native/SummarizerDALImpl.cpp b/mllib-dal/src/main/native/SummarizerDALImpl.cpp index 6428c2696..3e93cd821 100644 --- a/mllib-dal/src/main/native/SummarizerDALImpl.cpp +++ b/mllib-dal/src/main/native/SummarizerDALImpl.cpp @@ -22,7 +22,7 @@ #endif #include "OneCCL.h" -#include "org_apache_spark_mllib_stat_SummarizerDALImpl.h" +#include "com_intel_oap_mllib_stat_SummarizerDALImpl.h" #include "service.h" using namespace std; @@ -31,220 +31,221 @@ using namespace daal::algorithms; typedef double algorithmFPType; /* Algorithm floating-point type */ -static void summarizer_compute(JNIEnv *env, - jobject obj, - int rankId, - ccl::communicator &comm, - const NumericTablePtr &pData, - size_t nBlocks, - jobject resultObj) { - using daal::byte; - auto t1 = std::chrono::high_resolution_clock::now(); - - const bool isRoot = (rankId == ccl_root); - - low_order_moments::Distributed localAlgorithm; - - /* Set the input data set to the algorithm */ - localAlgorithm.input.set(low_order_moments::data, pData); - - /* Compute low_order_moments */ - localAlgorithm.compute(); - - auto t2 = std::chrono::high_resolution_clock::now(); - auto duration = - std::chrono::duration_cast(t2 - t1).count(); - std::cout << "low_order_moments (native): local step took " << duration << " secs" - << std::endl; - - t1 = std::chrono::high_resolution_clock::now(); - - /* Serialize partial results required by step 2 */ - services::SharedPtr serializedData; - InputDataArchive dataArch; - localAlgorithm.getPartialResult()->serialize(dataArch); - size_t perNodeArchLength = dataArch.getSizeOfArchive(); - - serializedData = - services::SharedPtr(new byte[perNodeArchLength * nBlocks]); - - byte *nodeResults = new byte[perNodeArchLength]; - dataArch.copyArchiveToArray(nodeResults, perNodeArchLength); - std::vector aReceiveCount(comm.size(), - perNodeArchLength); // 4 x "14016" - - /* Transfer partial results to step 2 on the root node */ - ccl::gather((int8_t *)nodeResults, perNodeArchLength, - (int8_t *)(serializedData.get()), perNodeArchLength, comm) - .wait(); - t2 = std::chrono::high_resolution_clock::now(); - - duration = - std::chrono::duration_cast(t2 - t1).count(); - std::cout << "low_order_moments (native): ccl_gather took " << duration << " secs" - << std::endl; - if (isRoot) { - auto t1 = std::chrono::high_resolution_clock::now(); - /* Create an algorithm to compute covariance on the master node */ - low_order_moments::Distributed masterAlgorithm; - - for (size_t i = 0; i < nBlocks; i++) { - /* Deserialize partial results from step 1 */ - OutputDataArchive dataArch(serializedData.get() + - perNodeArchLength * i, - perNodeArchLength); - - low_order_moments::PartialResultPtr dataForStep2FromStep1(new low_order_moments::PartialResult()); - dataForStep2FromStep1->deserialize(dataArch); - - /* Set local partial results as input for the master-node algorithm - */ - masterAlgorithm.input.add(low_order_moments::partialResults, - dataForStep2FromStep1); - } - - /* Set the parameter to choose the type of the output matrix */ - masterAlgorithm.parameter.estimatesToCompute = low_order_moments::estimatesAll; - - /* Merge and finalizeCompute covariance decomposition on the master node */ - masterAlgorithm.compute(); - masterAlgorithm.finalizeCompute(); - - /* Retrieve the algorithm results */ - low_order_moments::ResultPtr result = masterAlgorithm.getResult(); - auto t2 = std::chrono::high_resolution_clock::now(); - auto duration = - std::chrono::duration_cast(t2 - t1).count(); - std::cout << "low_order_moments (native): master step took " << duration << " secs" - << std::endl; - - /* Print the results */ - printNumericTable(result->get(low_order_moments::mean), - "low_order_moments first 20 columns of " - "Mean :", - 1, 20); - printNumericTable(result->get(low_order_moments::variance), - "low_order_moments first 20 columns of " - "Variance :", - 1, 20); - printNumericTable(result->get(low_order_moments::minimum), - "low_order_moments first 20 columns of " - "Minimum :", - 1, 20); - printNumericTable(result->get(low_order_moments::maximum), - "low_order_moments first 20 columns of " - "Maximum :", - 1, 20); - printNumericTable(result->get(low_order_moments::sum), - "low_order_moments first 20 columns of " - "Sum :", - 1, 20); - printNumericTable(result->get(low_order_moments::sumSquares), - "low_order_moments first 20 columns of " - "SumSquares :", - 1, 20); - printNumericTable(result->get(low_order_moments::sumSquaresCentered), - "low_order_moments first 20 columns of " - "SumSquaresCentered :", - 1, 20); - printNumericTable(result->get(low_order_moments::secondOrderRawMoment), - "low_order_moments first 20 columns of " - "SecondOrderRawMoment :", - 1, 20); - printNumericTable(result->get(low_order_moments::standardDeviation), - "low_order_moments first 20 columns of " - "StandardDeviation :", - 1, 20); - printNumericTable(result->get(low_order_moments::variation), - "low_order_moments first 20 columns of " - "Variation :", - 1, 20); +static void summarizer_compute(JNIEnv *env, jobject obj, int rankId, + ccl::communicator &comm, + const NumericTablePtr &pData, size_t nBlocks, + jobject resultObj) { + using daal::byte; + auto t1 = std::chrono::high_resolution_clock::now(); + + const bool isRoot = (rankId == ccl_root); + + low_order_moments::Distributed localAlgorithm; + + /* Set the input data set to the algorithm */ + localAlgorithm.input.set(low_order_moments::data, pData); + + /* Compute low_order_moments */ + localAlgorithm.compute(); + + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): local step took " << duration + << " secs" << std::endl; + + t1 = std::chrono::high_resolution_clock::now(); + + /* Serialize partial results required by step 2 */ + services::SharedPtr serializedData; + InputDataArchive dataArch; + localAlgorithm.getPartialResult()->serialize(dataArch); + size_t perNodeArchLength = dataArch.getSizeOfArchive(); + + serializedData = + services::SharedPtr(new byte[perNodeArchLength * nBlocks]); + + byte *nodeResults = new byte[perNodeArchLength]; + dataArch.copyArchiveToArray(nodeResults, perNodeArchLength); + std::vector aReceiveCount(comm.size(), + perNodeArchLength); // 4 x "14016" + + /* Transfer partial results to step 2 on the root node */ + ccl::gather((int8_t *)nodeResults, perNodeArchLength, + (int8_t *)(serializedData.get()), perNodeArchLength, comm) + .wait(); + t2 = std::chrono::high_resolution_clock::now(); + + duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): ccl_gather took " << duration + << " secs" << std::endl; + if (isRoot) { + auto t1 = std::chrono::high_resolution_clock::now(); + /* Create an algorithm to compute covariance on the master node */ + low_order_moments::Distributed + masterAlgorithm; + + for (size_t i = 0; i < nBlocks; i++) { + /* Deserialize partial results from step 1 */ + OutputDataArchive dataArch(serializedData.get() + + perNodeArchLength * i, + perNodeArchLength); + + low_order_moments::PartialResultPtr dataForStep2FromStep1( + new low_order_moments::PartialResult()); + dataForStep2FromStep1->deserialize(dataArch); + + /* Set local partial results as input for the master-node algorithm + */ + masterAlgorithm.input.add(low_order_moments::partialResults, + dataForStep2FromStep1); + } + + /* Set the parameter to choose the type of the output matrix */ + masterAlgorithm.parameter.estimatesToCompute = + low_order_moments::estimatesAll; + + /* Merge and finalizeCompute covariance decomposition on the master node + */ + masterAlgorithm.compute(); + masterAlgorithm.finalizeCompute(); + + /* Retrieve the algorithm results */ + low_order_moments::ResultPtr result = masterAlgorithm.getResult(); + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): master step took " << duration + << " secs" << std::endl; + + /* Print the results */ + printNumericTable(result->get(low_order_moments::mean), + "low_order_moments first 20 columns of " + "Mean :", + 1, 20); + printNumericTable(result->get(low_order_moments::variance), + "low_order_moments first 20 columns of " + "Variance :", + 1, 20); + printNumericTable(result->get(low_order_moments::minimum), + "low_order_moments first 20 columns of " + "Minimum :", + 1, 20); + printNumericTable(result->get(low_order_moments::maximum), + "low_order_moments first 20 columns of " + "Maximum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sum), + "low_order_moments first 20 columns of " + "Sum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquares), + "low_order_moments first 20 columns of " + "SumSquares :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquaresCentered), + "low_order_moments first 20 columns of " + "SumSquaresCentered :", + 1, 20); + printNumericTable(result->get(low_order_moments::secondOrderRawMoment), + "low_order_moments first 20 columns of " + "SecondOrderRawMoment :", + 1, 20); + printNumericTable(result->get(low_order_moments::standardDeviation), + "low_order_moments first 20 columns of " + "StandardDeviation :", + 1, 20); + printNumericTable(result->get(low_order_moments::variation), + "low_order_moments first 20 columns of " + "Variation :", + 1, 20); + // Return all covariance & mean jclass clazz = env->GetObjectClass(resultObj); - // Get Field references - jfieldID meanNumericTableField = - env->GetFieldID(clazz, "meanNumericTable", "J"); - jfieldID varianceNumericTableField = - env->GetFieldID(clazz, "varianceNumericTable", "J"); - jfieldID minimumNumericTableField = - env->GetFieldID(clazz, "minimumNumericTable", "J"); - jfieldID maximumNumericTableField = - env->GetFieldID(clazz, "maximumNumericTable", "J"); - - NumericTablePtr *mean = - new NumericTablePtr(result->get(low_order_moments::mean)); - NumericTablePtr *variance = - new NumericTablePtr(result->get(low_order_moments::variance)); - NumericTablePtr *max = - new NumericTablePtr(result->get(low_order_moments::maximum)); - NumericTablePtr *min = - new NumericTablePtr(result->get(low_order_moments::minimum)); - - env->SetLongField(resultObj, meanNumericTableField, (jlong)mean); - env->SetLongField(resultObj, varianceNumericTableField, (jlong)variance); - env->SetLongField(resultObj, maximumNumericTableField, (jlong)max); - env->SetLongField(resultObj, minimumNumericTableField, (jlong)min); - - } + // Get Field references + jfieldID meanNumericTableField = + env->GetFieldID(clazz, "meanNumericTable", "J"); + jfieldID varianceNumericTableField = + env->GetFieldID(clazz, "varianceNumericTable", "J"); + jfieldID minimumNumericTableField = + env->GetFieldID(clazz, "minimumNumericTable", "J"); + jfieldID maximumNumericTableField = + env->GetFieldID(clazz, "maximumNumericTable", "J"); + + NumericTablePtr *mean = + new NumericTablePtr(result->get(low_order_moments::mean)); + NumericTablePtr *variance = + new NumericTablePtr(result->get(low_order_moments::variance)); + NumericTablePtr *max = + new NumericTablePtr(result->get(low_order_moments::maximum)); + NumericTablePtr *min = + new NumericTablePtr(result->get(low_order_moments::minimum)); + + env->SetLongField(resultObj, meanNumericTableField, (jlong)mean); + env->SetLongField(resultObj, varianceNumericTableField, + (jlong)variance); + env->SetLongField(resultObj, maximumNumericTableField, (jlong)max); + env->SetLongField(resultObj, minimumNumericTableField, (jlong)min); + } } /* - * Class: org_apache_spark_mllib_stat_CorrelationDALImpl + * Class: com_intel_oap_mllib_stat_CorrelationDALImpl * Method: cCorrelationTrainDAL * Signature: (JJDDIILorg/apache/spark/ml/stat/CorrelationResult;)J */ JNIEXPORT jlong JNICALL -Java_org_apache_spark_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL( - JNIEnv *env, jobject obj, jlong pNumTabData, - jint executor_num, jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, jobject resultObj) { +Java_com_intel_oap_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL( + JNIEnv *env, jobject obj, jlong pNumTabData, jint executor_num, + jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, + jobject resultObj) { ccl::communicator &comm = getComm(); size_t rankId = comm.rank(); - std::cout << " rankId : " << rankId << " ! " - << std::endl; + std::cout << " rankId : " << rankId << " ! " << std::endl; const size_t nBlocks = executor_num; NumericTablePtr pData = *((NumericTablePtr *)pNumTabData); - #ifdef CPU_GPU_PROFILE +#ifdef CPU_GPU_PROFILE - if (use_gpu) { - int n_gpu = env->GetArrayLength(gpu_idx_array); - cout << "oneDAL (native): use GPU kernels with " << n_gpu << " GPU(s)" - << endl; + if (use_gpu) { - jint *gpu_indices = env->GetIntArrayElements(gpu_idx_array, 0); + int n_gpu = env->GetArrayLength(gpu_idx_array); + cout << "oneDAL (native): use GPU kernels with " << n_gpu << " GPU(s)" + << endl; - int size = comm.size(); - auto assigned_gpu = - getAssignedGPU(comm, size, rankId, gpu_indices, n_gpu); + jint *gpu_indices = env->GetIntArrayElements(gpu_idx_array, 0); - // Set SYCL context - cl::sycl::queue queue(assigned_gpu); - daal::services::SyclExecutionContext ctx(queue); - daal::services::Environment::getInstance()->setDefaultExecutionContext( - ctx); + int size = comm.size(); + auto assigned_gpu = + getAssignedGPU(comm, size, rankId, gpu_indices, n_gpu); - summarizer_compute( - env, obj, rankId, comm, pData, nBlocks, resultObj); + // Set SYCL context + cl::sycl::queue queue(assigned_gpu); + daal::services::SyclExecutionContext ctx(queue); + daal::services::Environment::getInstance()->setDefaultExecutionContext( + ctx); - env->ReleaseIntArrayElements(gpu_idx_array, gpu_indices, 0); - } else - #endif - { - // Set number of threads for oneDAL to use for each rank - services::Environment::getInstance()->setNumberOfThreads(executor_cores); + summarizer_compute(env, obj, rankId, comm, pData, nBlocks, resultObj); + env->ReleaseIntArrayElements(gpu_idx_array, gpu_indices, 0); + } else +#endif + { + // 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; + int nThreadsNew = + services::Environment::getInstance()->getNumberOfThreads(); + cout << "oneDAL (native): Number of CPU threads used: " << nThreadsNew + << endl; - summarizer_compute( - env, obj, rankId, comm, pData, nBlocks, resultObj); - } + summarizer_compute(env, obj, rankId, comm, pData, nBlocks, resultObj); + } - return 0; + return 0; } diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h new file mode 100644 index 000000000..60669f7cd --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_stat_SummarizerDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_stat_SummarizerDALImpl +#define _Included_com_intel_oap_mllib_stat_SummarizerDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_stat_SummarizerDALImpl + * Method: cSummarizerTrainDAL + * Signature: (JIIZ[ILcom/intel/oap/mllib/stat/SummarizerResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL + (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h b/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h deleted file mode 100644 index 4f0681069..000000000 --- a/mllib-dal/src/main/native/javah/org_apache_spark_mllib_stat_SummarizerDALImpl.h +++ /dev/null @@ -1,21 +0,0 @@ -/* DO NOT EDIT THIS FILE - it is machine generated */ -#include -/* Header for class org_apache_spark_mllib_stat_SummarizerDALImpl */ - -#ifndef _Included_org_apache_spark_mllib_stat_SummarizerDALImpl -#define _Included_org_apache_spark_mllib_stat_SummarizerDALImpl -#ifdef __cplusplus -extern "C" { -#endif -/* - * Class: org_apache_spark_mllib_stat_SummarizerDALImpl - * Method: cSummarizerTrainDAL - * Signature: (JIIZ[ILorg/apache/spark/mllib/stat/SummarizerResult;)J - */ -JNIEXPORT jlong JNICALL Java_org_apache_spark_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL - (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); - -#ifdef __cplusplus -} -#endif -#endif diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala index f44ffdc0b..937740eb4 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala @@ -32,11 +32,7 @@ object Utils { } def isOAPEnabled(sc: SparkContext): Boolean = { - return sc.conf.getBoolean("spark.oap.mllib.enabled", true) - } - - def isOAPEnabled(sc: SparkContext): Boolean = { - return sc.conf.getBoolean("spark.oap.mllib.enabled", true) + return sc.getConf.getBoolean("spark.oap.mllib.enabled", true) } def getOneCCLIPPort(data: RDD[_]): String = { diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala similarity index 70% rename from mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala rename to mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala index fb3a9ddfa..525285079 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/SummarizerDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala @@ -14,17 +14,17 @@ * limitations under the License. */ -package org.apache.spark.mllib.stat +package com.intel.oap.mllib.stat +import com.intel.oap.mllib.{OneCCL, OneDAL} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.util.Utils.getOneCCLIPPort -import org.apache.spark.ml.util.{OneCCL, OneDAL} -import org.apache.spark.mllib.stat.SummarizerResult +import org.apache.spark.ml.linalg.{Vector} import org.apache.spark.mllib.linalg.{Vectors => OldVectors} -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary => Summary} +import org.apache.spark.mllib.stat.{MultivariateStatisticalDALSummary, MultivariateStatisticalSummary => Summary} import org.apache.spark.rdd.RDD +import com.intel.oap.mllib.Utils.getOneCCLIPPort + class SummarizerDALImpl( val executorNum: Int, @@ -32,11 +32,10 @@ class SummarizerDALImpl( extends Serializable with Logging { def computeSummarizerMatrix(data: RDD[Vector]): Summary = { - val kvsIPPort = getOneCCLIPPort(data) val sparkContext = data.sparkContext - val useGPU = sparkContext.conf.getBoolean("spark.oap.mllib.useGPU", false) + val useGPU = sparkContext.getConf.getBoolean("spark.oap.mllib.useGPU", false) val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) @@ -73,10 +72,15 @@ class SummarizerDALImpl( val ret = if (OneCCL.isRoot()) { val convResultStartTime = System.nanoTime() - val meanArray= OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.meanNumericTable)) - val varianceArray = OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.varianceNumericTable)) - val maxrray= OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.maximumNumericTable)) - val minArray = OneDAL.numericTableToVectors(OneDAL.makeNumericTable(result.minimumNumericTable)) + val meanMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.meanNumericTable)) + val varianceMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.varianceNumericTable)) + val maxMatrix= OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.maximumNumericTable)) + val minMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.minimumNumericTable)) + + val meanVector = OldVectors.dense(meanMatrix.toArray) + val varianceVector = OldVectors.dense(varianceMatrix.toArray) + val maxVector = OldVectors.dense(maxMatrix.toArray) + val minVector = OldVectors.dense(minMatrix.toArray) val convResultEndTime = System.nanoTime() @@ -84,7 +88,7 @@ class SummarizerDALImpl( logInfo(s"SummarizerDAL result conversion took ${durationCovResult} secs") - Iterator((meanArray(0), varianceArray(0), maxrray(0), minArray(0))) + Iterator((meanVector, varianceVector, maxVector, minVector)) } else { Iterator.empty } @@ -102,8 +106,7 @@ class SummarizerDALImpl( val maxVector = results(0)._3 val minVector = results(0)._4 - val summary = new MultivariateStatisticalDALSummary(OldVectors.fromML(meanVector), OldVectors.fromML(varianceVector) - , OldVectors.fromML(maxVector), OldVectors.fromML(minVector)) + val summary = new MultivariateStatisticalDALSummary(meanVector, varianceVector, maxVector, minVector) summary } diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala new file mode 100644 index 000000000..6136abe4d --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala @@ -0,0 +1,43 @@ +/* + * 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 com.intel.oap.mllib.stat + +import org.apache.spark.{SPARK_VERSION, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset} + +import org.apache.spark.mllib.stat.spark320.{Statistics => SummarizerSpark320 } + +trait SummarizerShim extends Serializable with Logging { + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary + + } + +object SummarizerShim extends Logging { + def create(): SummarizerShim = { + logInfo(s"Loading Summarizer for Spark $SPARK_VERSION") + val summarizer = SPARK_VERSION match { + case "3.1.1" | "3.1.2" | "3.2.0" => new SummarizerSpark320() + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + summarizer + } +} + diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala similarity index 100% rename from mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala rename to mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..a4443335c --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import com.intel.oap.mllib.stat.{CorrelationShim, SummarizerShim} +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.stat.correlation.Correlations + + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val shim = SummarizerShim.create() + shim.colStats(X) + } + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala new file mode 100644 index 000000000..214d58d95 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.spark320 + +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.stat.{SummarizerDALImpl, SummarizerShim} +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import scala.annotation.varargs + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +class Statistics extends SummarizerShim{ + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[org.apache.spark.mllib.stat.MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } +} diff --git a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala deleted file mode 100644 index 7fdeec20c..000000000 --- a/mllib-dal/src/spark-3.0.0/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.stat - -import scala.annotation.varargs - -import org.apache.spark.annotation.Since -import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} -import org.apache.spark.ml.stat._ -import org.apache.spark.mllib.linalg.{Matrix, Vector} -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.stat.correlation.Correlations -import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, - KolmogorovSmirnovTestResult} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.Row -import org.apache.spark.ml.util._ - - - -/** - * API for statistical functions in MLlib. - */ -@Since("1.1.0") -object Statistics { - - /** - * Computes column-wise summary statistics for the input RDD[Vector]. - * - * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. - * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. - */ - @Since("1.1.0") - def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - X.sparkContext) - if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { - val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) - if (handlePersistence) { - X.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = X.map { - v => v.asML - } - val executor_num = Utils.sparkExecutorNum(X.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val summary = new SummarizerDALImpl(executor_num, executor_cores) - .computeSummarizerMatrix(rdd) - if (handlePersistence) { - X.unpersist() - } - summary - } else { - new RowMatrix(X).computeColumnSummaryStatistics() - } - } - - /** - * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. - * - * @param X an RDD containing vectors and weights for which column-wise summary statistics - * are to be computed. - * @return [[SummarizerBuffer]] object containing column-wise summary statistics. - */ - private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { - X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( - seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, - combOp = { case (c1, c2) => c1.merge(c2) }, - depth = 2 - ) - } - - /** - * Compute the Pearson correlation matrix for the input RDD of Vectors. - * Columns with 0 covariance produce NaN entries in the correlation matrix. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @return Pearson correlation matrix comparing columns in X. - */ - @Since("1.1.0") - def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) - - /** - * Compute the correlation matrix for the input RDD of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return Correlation matrix comparing columns in X. - * - * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to - * avoid recomputing the common lineage. - */ - @Since("1.1.0") - def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) - - /** - * Compute the Pearson correlation for the input RDDs. - * Returns NaN if either vector has 0 variance. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @return A Double containing the Pearson correlation between the two input RDD[Double]s - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) - - /** - * Compute the correlation for the input RDDs using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return A Double containing the correlation between the two input RDD[Double]s using the - * specified method. - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the - * expected distribution. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @param expected Vector containing the expected categorical counts/relative frequencies. - * `expected` is rescaled if the `expected` sum differs from the `observed` sum. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note The two input Vectors need to have the same size. - * `observed` cannot contain negative values. - * `expected` cannot contain nonpositive values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { - ChiSqTest.chiSquared(observed, expected) - } - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform - * distribution, with each category having an expected frequency of `1 / observed.size`. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note `observed` cannot contain negative values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) - - /** - * Conduct Pearson's independence test on the input contingency matrix, which cannot contain - * negative entries or columns or rows that sum up to 0. - * - * @param observed The contingency matrix (containing either counts or relative frequencies). - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - */ - @Since("1.1.0") - def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) - - /** - * Conduct Pearson's independence test for every feature against the label across the input RDD. - * For each feature, the (feature, label) pairs are converted into a contingency matrix for which - * the chi-squared statistic is computed. All label and feature values must be categorical. - * - * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. - * Real-valued features will be treated as categorical for each distinct value. - * @return an array containing the ChiSquaredTestResult for every feature against the label. - * The order of the elements in the returned array reflects the order of input features. - */ - @Since("1.1.0") - def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { - ChiSqTest.chiSquaredFeatures(data) - } - - /** - * Java-friendly version of `chiSqTest()` - */ - @Since("1.5.0") - def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) - - /** - * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a - * continuous distribution. By comparing the largest difference between the empirical cumulative - * distribution of the sample data and the theoretical distribution we can provide a test for the - * the null hypothesis that the sample data comes from that theoretical distribution. - * For more information on KS Test: - * @see - * Kolmogorov-Smirnov test (Wikipedia) - * - * @param data an `RDD[Double]` containing the sample of data to test - * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, cdf) - } - - /** - * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability - * distribution equality. Currently supports the normal distribution, taking as parameters - * the mean and standard deviation. - * (distName = "norm") - * @param data an `RDD[Double]` containing the sample of data to test - * @param distName a `String` name for a theoretical distribution - * @param params `Double*` specifying the parameters to be used for the theoretical distribution - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) - } - - /** - * Java-friendly version of `kolmogorovSmirnovTest()` - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest( - data: JavaDoubleRDD, - distName: String, - params: Double*): KolmogorovSmirnovTestResult = { - kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) - } -} diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala deleted file mode 100644 index f323197ce..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala +++ /dev/null @@ -1,65 +0,0 @@ -package org.apache.spark.mllib.stat - -import org.apache.spark.mllib.linalg.Vector - -class MultivariateStatisticalDALSummary ( - val meanVector: Vector, - val varianceVector: Vector, - val maxVector: Vector, - val minVector: Vector) - extends MultivariateStatisticalSummary with Serializable { - - /** - * Sample mean vector. - */ - override def mean: Vector = { - meanVector - } - - /** - * Sample variance vector. Should return a zero vector if the sample size is 1. - */ - override def variance: Vector = { - varianceVector - } - - /** - * Sample size. - */ - override def count: Long = 0 - - /** - * Sum of weights. - */ - override def weightSum: Double = 0.0 - - /** - * Number of nonzero elements (including explicitly presented zero values) in each column. - */ - override def numNonzeros: Vector = null - - /** - * Maximum value of each column. - */ - override def max: Vector = { - maxVector - } - - /** - * Minimum value of each column. - */ - override def min: Vector = { - minVector - } - - /** - * Euclidean magnitude of each column - */ - override def normL2: Vector = null - - /** - * L1 norm of each column - */ - override def normL1: Vector = null - -} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala deleted file mode 100644 index 7fdeec20c..000000000 --- a/mllib-dal/src/spark-3.0.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.stat - -import scala.annotation.varargs - -import org.apache.spark.annotation.Since -import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} -import org.apache.spark.ml.stat._ -import org.apache.spark.mllib.linalg.{Matrix, Vector} -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.stat.correlation.Correlations -import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, - KolmogorovSmirnovTestResult} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.Row -import org.apache.spark.ml.util._ - - - -/** - * API for statistical functions in MLlib. - */ -@Since("1.1.0") -object Statistics { - - /** - * Computes column-wise summary statistics for the input RDD[Vector]. - * - * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. - * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. - */ - @Since("1.1.0") - def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - X.sparkContext) - if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { - val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) - if (handlePersistence) { - X.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = X.map { - v => v.asML - } - val executor_num = Utils.sparkExecutorNum(X.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val summary = new SummarizerDALImpl(executor_num, executor_cores) - .computeSummarizerMatrix(rdd) - if (handlePersistence) { - X.unpersist() - } - summary - } else { - new RowMatrix(X).computeColumnSummaryStatistics() - } - } - - /** - * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. - * - * @param X an RDD containing vectors and weights for which column-wise summary statistics - * are to be computed. - * @return [[SummarizerBuffer]] object containing column-wise summary statistics. - */ - private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { - X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( - seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, - combOp = { case (c1, c2) => c1.merge(c2) }, - depth = 2 - ) - } - - /** - * Compute the Pearson correlation matrix for the input RDD of Vectors. - * Columns with 0 covariance produce NaN entries in the correlation matrix. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @return Pearson correlation matrix comparing columns in X. - */ - @Since("1.1.0") - def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) - - /** - * Compute the correlation matrix for the input RDD of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return Correlation matrix comparing columns in X. - * - * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to - * avoid recomputing the common lineage. - */ - @Since("1.1.0") - def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) - - /** - * Compute the Pearson correlation for the input RDDs. - * Returns NaN if either vector has 0 variance. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @return A Double containing the Pearson correlation between the two input RDD[Double]s - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) - - /** - * Compute the correlation for the input RDDs using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return A Double containing the correlation between the two input RDD[Double]s using the - * specified method. - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the - * expected distribution. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @param expected Vector containing the expected categorical counts/relative frequencies. - * `expected` is rescaled if the `expected` sum differs from the `observed` sum. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note The two input Vectors need to have the same size. - * `observed` cannot contain negative values. - * `expected` cannot contain nonpositive values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { - ChiSqTest.chiSquared(observed, expected) - } - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform - * distribution, with each category having an expected frequency of `1 / observed.size`. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note `observed` cannot contain negative values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) - - /** - * Conduct Pearson's independence test on the input contingency matrix, which cannot contain - * negative entries or columns or rows that sum up to 0. - * - * @param observed The contingency matrix (containing either counts or relative frequencies). - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - */ - @Since("1.1.0") - def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) - - /** - * Conduct Pearson's independence test for every feature against the label across the input RDD. - * For each feature, the (feature, label) pairs are converted into a contingency matrix for which - * the chi-squared statistic is computed. All label and feature values must be categorical. - * - * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. - * Real-valued features will be treated as categorical for each distinct value. - * @return an array containing the ChiSquaredTestResult for every feature against the label. - * The order of the elements in the returned array reflects the order of input features. - */ - @Since("1.1.0") - def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { - ChiSqTest.chiSquaredFeatures(data) - } - - /** - * Java-friendly version of `chiSqTest()` - */ - @Since("1.5.0") - def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) - - /** - * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a - * continuous distribution. By comparing the largest difference between the empirical cumulative - * distribution of the sample data and the theoretical distribution we can provide a test for the - * the null hypothesis that the sample data comes from that theoretical distribution. - * For more information on KS Test: - * @see - * Kolmogorov-Smirnov test (Wikipedia) - * - * @param data an `RDD[Double]` containing the sample of data to test - * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, cdf) - } - - /** - * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability - * distribution equality. Currently supports the normal distribution, taking as parameters - * the mean and standard deviation. - * (distName = "norm") - * @param data an `RDD[Double]` containing the sample of data to test - * @param distName a `String` name for a theoretical distribution - * @param params `Double*` specifying the parameters to be used for the theoretical distribution - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) - } - - /** - * Java-friendly version of `kolmogorovSmirnovTest()` - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest( - data: JavaDoubleRDD, - distName: String, - params: Double*): KolmogorovSmirnovTestResult = { - kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) - } -} diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala deleted file mode 100644 index f323197ce..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala +++ /dev/null @@ -1,65 +0,0 @@ -package org.apache.spark.mllib.stat - -import org.apache.spark.mllib.linalg.Vector - -class MultivariateStatisticalDALSummary ( - val meanVector: Vector, - val varianceVector: Vector, - val maxVector: Vector, - val minVector: Vector) - extends MultivariateStatisticalSummary with Serializable { - - /** - * Sample mean vector. - */ - override def mean: Vector = { - meanVector - } - - /** - * Sample variance vector. Should return a zero vector if the sample size is 1. - */ - override def variance: Vector = { - varianceVector - } - - /** - * Sample size. - */ - override def count: Long = 0 - - /** - * Sum of weights. - */ - override def weightSum: Double = 0.0 - - /** - * Number of nonzero elements (including explicitly presented zero values) in each column. - */ - override def numNonzeros: Vector = null - - /** - * Maximum value of each column. - */ - override def max: Vector = { - maxVector - } - - /** - * Minimum value of each column. - */ - override def min: Vector = { - minVector - } - - /** - * Euclidean magnitude of each column - */ - override def normL2: Vector = null - - /** - * L1 norm of each column - */ - override def normL1: Vector = null - -} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala deleted file mode 100644 index 7fdeec20c..000000000 --- a/mllib-dal/src/spark-3.0.2/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.stat - -import scala.annotation.varargs - -import org.apache.spark.annotation.Since -import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} -import org.apache.spark.ml.stat._ -import org.apache.spark.mllib.linalg.{Matrix, Vector} -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.stat.correlation.Correlations -import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, - KolmogorovSmirnovTestResult} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.Row -import org.apache.spark.ml.util._ - - - -/** - * API for statistical functions in MLlib. - */ -@Since("1.1.0") -object Statistics { - - /** - * Computes column-wise summary statistics for the input RDD[Vector]. - * - * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. - * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. - */ - @Since("1.1.0") - def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - X.sparkContext) - if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { - val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) - if (handlePersistence) { - X.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = X.map { - v => v.asML - } - val executor_num = Utils.sparkExecutorNum(X.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val summary = new SummarizerDALImpl(executor_num, executor_cores) - .computeSummarizerMatrix(rdd) - if (handlePersistence) { - X.unpersist() - } - summary - } else { - new RowMatrix(X).computeColumnSummaryStatistics() - } - } - - /** - * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. - * - * @param X an RDD containing vectors and weights for which column-wise summary statistics - * are to be computed. - * @return [[SummarizerBuffer]] object containing column-wise summary statistics. - */ - private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { - X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( - seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, - combOp = { case (c1, c2) => c1.merge(c2) }, - depth = 2 - ) - } - - /** - * Compute the Pearson correlation matrix for the input RDD of Vectors. - * Columns with 0 covariance produce NaN entries in the correlation matrix. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @return Pearson correlation matrix comparing columns in X. - */ - @Since("1.1.0") - def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) - - /** - * Compute the correlation matrix for the input RDD of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return Correlation matrix comparing columns in X. - * - * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to - * avoid recomputing the common lineage. - */ - @Since("1.1.0") - def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) - - /** - * Compute the Pearson correlation for the input RDDs. - * Returns NaN if either vector has 0 variance. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @return A Double containing the Pearson correlation between the two input RDD[Double]s - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) - - /** - * Compute the correlation for the input RDDs using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return A Double containing the correlation between the two input RDD[Double]s using the - * specified method. - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the - * expected distribution. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @param expected Vector containing the expected categorical counts/relative frequencies. - * `expected` is rescaled if the `expected` sum differs from the `observed` sum. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note The two input Vectors need to have the same size. - * `observed` cannot contain negative values. - * `expected` cannot contain nonpositive values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { - ChiSqTest.chiSquared(observed, expected) - } - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform - * distribution, with each category having an expected frequency of `1 / observed.size`. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note `observed` cannot contain negative values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) - - /** - * Conduct Pearson's independence test on the input contingency matrix, which cannot contain - * negative entries or columns or rows that sum up to 0. - * - * @param observed The contingency matrix (containing either counts or relative frequencies). - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - */ - @Since("1.1.0") - def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) - - /** - * Conduct Pearson's independence test for every feature against the label across the input RDD. - * For each feature, the (feature, label) pairs are converted into a contingency matrix for which - * the chi-squared statistic is computed. All label and feature values must be categorical. - * - * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. - * Real-valued features will be treated as categorical for each distinct value. - * @return an array containing the ChiSquaredTestResult for every feature against the label. - * The order of the elements in the returned array reflects the order of input features. - */ - @Since("1.1.0") - def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { - ChiSqTest.chiSquaredFeatures(data) - } - - /** - * Java-friendly version of `chiSqTest()` - */ - @Since("1.5.0") - def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) - - /** - * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a - * continuous distribution. By comparing the largest difference between the empirical cumulative - * distribution of the sample data and the theoretical distribution we can provide a test for the - * the null hypothesis that the sample data comes from that theoretical distribution. - * For more information on KS Test: - * @see - * Kolmogorov-Smirnov test (Wikipedia) - * - * @param data an `RDD[Double]` containing the sample of data to test - * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, cdf) - } - - /** - * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability - * distribution equality. Currently supports the normal distribution, taking as parameters - * the mean and standard deviation. - * (distName = "norm") - * @param data an `RDD[Double]` containing the sample of data to test - * @param distName a `String` name for a theoretical distribution - * @param params `Double*` specifying the parameters to be used for the theoretical distribution - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) - } - - /** - * Java-friendly version of `kolmogorovSmirnovTest()` - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest( - data: JavaDoubleRDD, - distName: String, - params: Double*): KolmogorovSmirnovTestResult = { - kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) - } -} diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala deleted file mode 100644 index f323197ce..000000000 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala +++ /dev/null @@ -1,65 +0,0 @@ -package org.apache.spark.mllib.stat - -import org.apache.spark.mllib.linalg.Vector - -class MultivariateStatisticalDALSummary ( - val meanVector: Vector, - val varianceVector: Vector, - val maxVector: Vector, - val minVector: Vector) - extends MultivariateStatisticalSummary with Serializable { - - /** - * Sample mean vector. - */ - override def mean: Vector = { - meanVector - } - - /** - * Sample variance vector. Should return a zero vector if the sample size is 1. - */ - override def variance: Vector = { - varianceVector - } - - /** - * Sample size. - */ - override def count: Long = 0 - - /** - * Sum of weights. - */ - override def weightSum: Double = 0.0 - - /** - * Number of nonzero elements (including explicitly presented zero values) in each column. - */ - override def numNonzeros: Vector = null - - /** - * Maximum value of each column. - */ - override def max: Vector = { - maxVector - } - - /** - * Minimum value of each column. - */ - override def min: Vector = { - minVector - } - - /** - * Euclidean magnitude of each column - */ - override def normL2: Vector = null - - /** - * L1 norm of each column - */ - override def normL1: Vector = null - -} \ No newline at end of file diff --git a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala deleted file mode 100644 index 7fdeec20c..000000000 --- a/mllib-dal/src/spark-3.1.1/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.stat - -import scala.annotation.varargs - -import org.apache.spark.annotation.Since -import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} -import org.apache.spark.ml.stat._ -import org.apache.spark.mllib.linalg.{Matrix, Vector} -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.stat.correlation.Correlations -import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, - KolmogorovSmirnovTestResult} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.Row -import org.apache.spark.ml.util._ - - - -/** - * API for statistical functions in MLlib. - */ -@Since("1.1.0") -object Statistics { - - /** - * Computes column-wise summary statistics for the input RDD[Vector]. - * - * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. - * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. - */ - @Since("1.1.0") - def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { - val isPlatformSupported = Utils.checkClusterPlatformCompatibility( - X.sparkContext) - if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { - val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) - if (handlePersistence) { - X.persist(StorageLevel.MEMORY_AND_DISK) - } - val rdd = X.map { - v => v.asML - } - val executor_num = Utils.sparkExecutorNum(X.sparkContext) - val executor_cores = Utils.sparkExecutorCores() - val summary = new SummarizerDALImpl(executor_num, executor_cores) - .computeSummarizerMatrix(rdd) - if (handlePersistence) { - X.unpersist() - } - summary - } else { - new RowMatrix(X).computeColumnSummaryStatistics() - } - } - - /** - * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. - * - * @param X an RDD containing vectors and weights for which column-wise summary statistics - * are to be computed. - * @return [[SummarizerBuffer]] object containing column-wise summary statistics. - */ - private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { - X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( - seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, - combOp = { case (c1, c2) => c1.merge(c2) }, - depth = 2 - ) - } - - /** - * Compute the Pearson correlation matrix for the input RDD of Vectors. - * Columns with 0 covariance produce NaN entries in the correlation matrix. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @return Pearson correlation matrix comparing columns in X. - */ - @Since("1.1.0") - def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) - - /** - * Compute the correlation matrix for the input RDD of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param X an RDD[Vector] for which the correlation matrix is to be computed. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return Correlation matrix comparing columns in X. - * - * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to - * avoid recomputing the common lineage. - */ - @Since("1.1.0") - def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) - - /** - * Compute the Pearson correlation for the input RDDs. - * Returns NaN if either vector has 0 variance. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @return A Double containing the Pearson correlation between the two input RDD[Double]s - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) - - /** - * Compute the correlation for the input RDDs using the specified method. - * Methods currently supported: `pearson` (default), `spearman`. - * - * @param x RDD[Double] of the same cardinality as y. - * @param y RDD[Double] of the same cardinality as x. - * @param method String specifying the method to use for computing correlation. - * Supported: `pearson` (default), `spearman` - * @return A Double containing the correlation between the two input RDD[Double]s using the - * specified method. - * - * @note The two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - */ - @Since("1.1.0") - def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) - - /** - * Java-friendly version of `corr()` - */ - @Since("1.4.1") - def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = - corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the - * expected distribution. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @param expected Vector containing the expected categorical counts/relative frequencies. - * `expected` is rescaled if the `expected` sum differs from the `observed` sum. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note The two input Vectors need to have the same size. - * `observed` cannot contain negative values. - * `expected` cannot contain nonpositive values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { - ChiSqTest.chiSquared(observed, expected) - } - - /** - * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform - * distribution, with each category having an expected frequency of `1 / observed.size`. - * - * @param observed Vector containing the observed categorical counts/relative frequencies. - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - * - * @note `observed` cannot contain negative values. - */ - @Since("1.1.0") - def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) - - /** - * Conduct Pearson's independence test on the input contingency matrix, which cannot contain - * negative entries or columns or rows that sum up to 0. - * - * @param observed The contingency matrix (containing either counts or relative frequencies). - * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, - * the method used, and the null hypothesis. - */ - @Since("1.1.0") - def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) - - /** - * Conduct Pearson's independence test for every feature against the label across the input RDD. - * For each feature, the (feature, label) pairs are converted into a contingency matrix for which - * the chi-squared statistic is computed. All label and feature values must be categorical. - * - * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. - * Real-valued features will be treated as categorical for each distinct value. - * @return an array containing the ChiSquaredTestResult for every feature against the label. - * The order of the elements in the returned array reflects the order of input features. - */ - @Since("1.1.0") - def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { - ChiSqTest.chiSquaredFeatures(data) - } - - /** - * Java-friendly version of `chiSqTest()` - */ - @Since("1.5.0") - def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) - - /** - * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a - * continuous distribution. By comparing the largest difference between the empirical cumulative - * distribution of the sample data and the theoretical distribution we can provide a test for the - * the null hypothesis that the sample data comes from that theoretical distribution. - * For more information on KS Test: - * @see - * Kolmogorov-Smirnov test (Wikipedia) - * - * @param data an `RDD[Double]` containing the sample of data to test - * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, cdf) - } - - /** - * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability - * distribution equality. Currently supports the normal distribution, taking as parameters - * the mean and standard deviation. - * (distName = "norm") - * @param data an `RDD[Double]` containing the sample of data to test - * @param distName a `String` name for a theoretical distribution - * @param params `Double*` specifying the parameters to be used for the theoretical distribution - * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test - * statistic, p-value, and null hypothesis. - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) - : KolmogorovSmirnovTestResult = { - KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) - } - - /** - * Java-friendly version of `kolmogorovSmirnovTest()` - */ - @Since("1.5.0") - @varargs - def kolmogorovSmirnovTest( - data: JavaDoubleRDD, - distName: String, - params: Double*): KolmogorovSmirnovTestResult = { - kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) - } -} From 86bc2b27b2c539c4a5cc6be2237aa8024f63f97e Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Thu, 25 Nov 2021 13:56:26 +0800 Subject: [PATCH 04/13] 1. add Summarizer example and CI 2. refactor Low-Order Moments Signed-off-by: minmingzhu --- examples/build-all-scala.sh | 2 +- examples/run-all-scala.sh | 2 +- examples/summarizer/IntelGpuResourceFile.json | 1 + examples/summarizer/build.sh | 3 + examples/summarizer/pom.xml | 94 +++++++ examples/summarizer/run-gpu-standalone.sh | 41 +++ examples/summarizer/run.sh | 30 +++ .../ml/SummaryStatisticsExample.scala | 67 +++++ .../src/main/native/SummarizerDALImpl.cpp | 251 ++++++++++++++++++ mllib-dal/src/main/native/build-jni.sh | 3 +- ...m_intel_oap_mllib_stat_SummarizerDALImpl.h | 21 ++ .../scala/com/intel/oap/mllib/Utils.scala | 4 + .../oap/mllib/stat/SummarizerDALImpl.scala | 121 +++++++++ .../intel/oap/mllib/stat/SummarizerShim.scala | 43 +++ .../MultivariateStatisticalDALSummary.scala | 65 +++++ .../apache/spark/mllib/stat/Statistics.scala | 119 +++++++++ .../mllib/stat/spark320/Statistics.scala | 73 +++++ 17 files changed, 937 insertions(+), 3 deletions(-) create mode 100644 examples/summarizer/IntelGpuResourceFile.json create mode 100644 examples/summarizer/build.sh create mode 100644 examples/summarizer/pom.xml create mode 100755 examples/summarizer/run-gpu-standalone.sh create mode 100644 examples/summarizer/run.sh create mode 100644 examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala create mode 100644 mllib-dal/src/main/native/SummarizerDALImpl.cpp create mode 100644 mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala create mode 100644 mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala diff --git a/examples/build-all-scala.sh b/examples/build-all-scala.sh index 883f87868..8babb9d97 100755 --- a/examples/build-all-scala.sh +++ b/examples/build-all-scala.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) +exampleDirs=(kmeans pca als naive-bayes linear-regression correlation summarizer) for dir in ${exampleDirs[*]} do diff --git a/examples/run-all-scala.sh b/examples/run-all-scala.sh index 269341848..04bab7f8a 100755 --- a/examples/run-all-scala.sh +++ b/examples/run-all-scala.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -exampleDirs=(kmeans pca als naive-bayes linear-regression correlation) +exampleDirs=(kmeans pca als naive-bayes linear-regression correlation summarizer) for dir in ${exampleDirs[*]} do diff --git a/examples/summarizer/IntelGpuResourceFile.json b/examples/summarizer/IntelGpuResourceFile.json new file mode 100644 index 000000000..4b5c3cc98 --- /dev/null +++ b/examples/summarizer/IntelGpuResourceFile.json @@ -0,0 +1 @@ +[{"id":{"componentName": "spark.worker","resourceName":"gpu"},"addresses":["0","1","2","3"]}] diff --git a/examples/summarizer/build.sh b/examples/summarizer/build.sh new file mode 100644 index 000000000..da373645b --- /dev/null +++ b/examples/summarizer/build.sh @@ -0,0 +1,3 @@ +#!/usr/bin/env bash + +mvn clean package diff --git a/examples/summarizer/pom.xml b/examples/summarizer/pom.xml new file mode 100644 index 000000000..51aa36355 --- /dev/null +++ b/examples/summarizer/pom.xml @@ -0,0 +1,94 @@ + + 4.0.0 + + com.intel.oap + oap-mllib-examples + 1.3.0 + jar + + SummaryStatisticsExample + https://github.com/oap-project/oap-mllib.git + + + UTF-8 + 1.3.0 + 2.12.10 + 2.12 + 3.1.1 + + + + + + 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} + + + + + org.apache.spark + spark-mllib_2.12 + ${spark.version} + + + + + + + + + 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/summarizer/run-gpu-standalone.sh b/examples/summarizer/run-gpu-standalone.sh new file mode 100755 index 000000000..853b479db --- /dev/null +++ b/examples/summarizer/run-gpu-standalone.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + +# Data file is from Spark Examples (data/mllib/sample_kmeans_data.txt) and put in examples/data +# The data file should be copied to $HDFS_ROOT before running examples + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar +APP_CLASS=org.apache.spark.examples.ml.SummaryStatisticsExample + +USE_GPU=true +RESOURCE_FILE=$PWD/IntelGpuResourceFile.json +WORKER_GPU_AMOUNT=4 +EXECUTOR_GPU_AMOUNT=1 +TASK_GPU_AMOUNT=1 + +# Should run in standalone mode +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --total-executor-cores $SPARK_TOTAL_CORES \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --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.oap.mllib.useGPU=$USE_GPU" \ + --conf "spark.worker.resourcesFile=$RESOURCE_FILE" \ + --conf "spark.worker.resource.gpu.amount=$WORKER_GPU_AMOUNT" \ + --conf "spark.executor.resource.gpu.amount=$EXECUTOR_GPU_AMOUNT" \ + --conf "spark.task.resource.gpu.amount=$TASK_GPU_AMOUNT" \ + --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 \ + 2>&1 | tee Summarizer-$(date +%m%d_%H_%M_%S).log + diff --git a/examples/summarizer/run.sh b/examples/summarizer/run.sh new file mode 100644 index 000000000..6a8bab7d3 --- /dev/null +++ b/examples/summarizer/run.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +source ../../conf/env.sh + + +APP_JAR=target/oap-mllib-examples-$OAP_MLLIB_VERSION.jar +APP_CLASS=org.apache.spark.examples.ml.SummaryStatisticsExample + +time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --total-executor-cores $SPARK_TOTAL_CORES \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --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" \ + --conf "spark.eventLog.enabled=true" \ + --conf "spark.eventLog.dir=hdfs://beaver:9000/spark-history-server" \ + --conf "spark.history.fs.logDirectory=hdfs://beaver:9000/spark-history-server" \ + --conf "spark.driver.log.dfsDir=/user/spark/driverLogs" \ + --jars $OAP_MLLIB_JAR \ + --class $APP_CLASS \ + $APP_JAR \ + 2>&1 | tee Summarizer-$(date +%m%d_%H_%M_%S).log \ No newline at end of file diff --git a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala new file mode 100644 index 000000000..a173e2782 --- /dev/null +++ b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala @@ -0,0 +1,67 @@ +/* + * 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 + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector => NewVector} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.SparkSession +import org.apache.spark.rdd.RDD + +// $example off$ + +object SummaryStatisticsExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("SummaryStatisticsExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = sc.parallelize( Seq( + Vectors.dense(5.308206,9.869278,1.018934,4.292158,6.081011,6.585723,2.411094,4.767308,-3.256320,-6.029562), + Vectors.dense(7.279464,0.390664,-9.619284,3.435376,-4.769490,-4.873188,-0.118791,-5.117316,-0.418655,-0.475422), + Vectors.dense(-6.615791,-6.191542,0.402459,-9.743521,-9.990568,9.105346,1.691312,-2.605659,9.534952,-7.829027), + Vectors.dense(-4.792007,-2.491098,-2.939393,8.086467,3.773812,-9.997300,0.222378,8.995244,-5.753282,6.091060), + Vectors.dense(7.700725,-6.414918,1.684476,-8.983361,4.284580,-9.017608,0.552379,-7.705741,2.589852,0.411561), + Vectors.dense(6.991900,-1.063721,9.321163,-0.429719,-2.167696,-1.736425,-0.919139,6.980681,-0.711914,3.414347), + Vectors.dense(5.794488,-1.062261,0.955322,0.389642,3.012921,-9.953994,-3.197309,3.992421,-6.935902,8.147622), + Vectors.dense(-2.486670,6.973242,-4.047004,-5.655629,5.081786,5.533859,7.821403,2.763313,-0.454056,6.554309), + Vectors.dense(2.204855,7.839522,7.381886,1.618749,-6.566877,7.584285,-8.355983,-5.501410,-8.191205,-2.608499), + Vectors.dense(-9.948613,-8.941953,-8.106389,4.863542,5.852806,-1.659259,6.342504,-8.190106,-3.110330,-7.484658), + )) + + val summary: MultivariateStatisticalSummary = Statistics.colStats(data) + println(summary.mean) // a dense vector containing the mean value for each column + println(summary.variance) // column-wise variance + println(summary.max) + println(summary.min) + println(summary.count) + println(summary.normL1) + println(summary.normL2) + println(summary.weightSum) + println(summary.numNonzeros) // number of nonzeros in each column + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/mllib-dal/src/main/native/SummarizerDALImpl.cpp b/mllib-dal/src/main/native/SummarizerDALImpl.cpp new file mode 100644 index 000000000..3e93cd821 --- /dev/null +++ b/mllib-dal/src/main/native/SummarizerDALImpl.cpp @@ -0,0 +1,251 @@ +/******************************************************************************* + * 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. + *******************************************************************************/ + +#include +#include + +#ifdef CPU_GPU_PROFILE +#include "GPU.h" +#endif + +#include "OneCCL.h" +#include "com_intel_oap_mllib_stat_SummarizerDALImpl.h" +#include "service.h" + +using namespace std; +using namespace daal; +using namespace daal::algorithms; + +typedef double algorithmFPType; /* Algorithm floating-point type */ + +static void summarizer_compute(JNIEnv *env, jobject obj, int rankId, + ccl::communicator &comm, + const NumericTablePtr &pData, size_t nBlocks, + jobject resultObj) { + using daal::byte; + auto t1 = std::chrono::high_resolution_clock::now(); + + const bool isRoot = (rankId == ccl_root); + + low_order_moments::Distributed localAlgorithm; + + /* Set the input data set to the algorithm */ + localAlgorithm.input.set(low_order_moments::data, pData); + + /* Compute low_order_moments */ + localAlgorithm.compute(); + + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): local step took " << duration + << " secs" << std::endl; + + t1 = std::chrono::high_resolution_clock::now(); + + /* Serialize partial results required by step 2 */ + services::SharedPtr serializedData; + InputDataArchive dataArch; + localAlgorithm.getPartialResult()->serialize(dataArch); + size_t perNodeArchLength = dataArch.getSizeOfArchive(); + + serializedData = + services::SharedPtr(new byte[perNodeArchLength * nBlocks]); + + byte *nodeResults = new byte[perNodeArchLength]; + dataArch.copyArchiveToArray(nodeResults, perNodeArchLength); + std::vector aReceiveCount(comm.size(), + perNodeArchLength); // 4 x "14016" + + /* Transfer partial results to step 2 on the root node */ + ccl::gather((int8_t *)nodeResults, perNodeArchLength, + (int8_t *)(serializedData.get()), perNodeArchLength, comm) + .wait(); + t2 = std::chrono::high_resolution_clock::now(); + + duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): ccl_gather took " << duration + << " secs" << std::endl; + if (isRoot) { + auto t1 = std::chrono::high_resolution_clock::now(); + /* Create an algorithm to compute covariance on the master node */ + low_order_moments::Distributed + masterAlgorithm; + + for (size_t i = 0; i < nBlocks; i++) { + /* Deserialize partial results from step 1 */ + OutputDataArchive dataArch(serializedData.get() + + perNodeArchLength * i, + perNodeArchLength); + + low_order_moments::PartialResultPtr dataForStep2FromStep1( + new low_order_moments::PartialResult()); + dataForStep2FromStep1->deserialize(dataArch); + + /* Set local partial results as input for the master-node algorithm + */ + masterAlgorithm.input.add(low_order_moments::partialResults, + dataForStep2FromStep1); + } + + /* Set the parameter to choose the type of the output matrix */ + masterAlgorithm.parameter.estimatesToCompute = + low_order_moments::estimatesAll; + + /* Merge and finalizeCompute covariance decomposition on the master node + */ + masterAlgorithm.compute(); + masterAlgorithm.finalizeCompute(); + + /* Retrieve the algorithm results */ + low_order_moments::ResultPtr result = masterAlgorithm.getResult(); + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = + std::chrono::duration_cast(t2 - t1).count(); + std::cout << "low_order_moments (native): master step took " << duration + << " secs" << std::endl; + + /* Print the results */ + printNumericTable(result->get(low_order_moments::mean), + "low_order_moments first 20 columns of " + "Mean :", + 1, 20); + printNumericTable(result->get(low_order_moments::variance), + "low_order_moments first 20 columns of " + "Variance :", + 1, 20); + printNumericTable(result->get(low_order_moments::minimum), + "low_order_moments first 20 columns of " + "Minimum :", + 1, 20); + printNumericTable(result->get(low_order_moments::maximum), + "low_order_moments first 20 columns of " + "Maximum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sum), + "low_order_moments first 20 columns of " + "Sum :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquares), + "low_order_moments first 20 columns of " + "SumSquares :", + 1, 20); + printNumericTable(result->get(low_order_moments::sumSquaresCentered), + "low_order_moments first 20 columns of " + "SumSquaresCentered :", + 1, 20); + printNumericTable(result->get(low_order_moments::secondOrderRawMoment), + "low_order_moments first 20 columns of " + "SecondOrderRawMoment :", + 1, 20); + printNumericTable(result->get(low_order_moments::standardDeviation), + "low_order_moments first 20 columns of " + "StandardDeviation :", + 1, 20); + printNumericTable(result->get(low_order_moments::variation), + "low_order_moments first 20 columns of " + "Variation :", + 1, 20); + + // Return all covariance & mean + jclass clazz = env->GetObjectClass(resultObj); + + // Get Field references + jfieldID meanNumericTableField = + env->GetFieldID(clazz, "meanNumericTable", "J"); + jfieldID varianceNumericTableField = + env->GetFieldID(clazz, "varianceNumericTable", "J"); + jfieldID minimumNumericTableField = + env->GetFieldID(clazz, "minimumNumericTable", "J"); + jfieldID maximumNumericTableField = + env->GetFieldID(clazz, "maximumNumericTable", "J"); + + NumericTablePtr *mean = + new NumericTablePtr(result->get(low_order_moments::mean)); + NumericTablePtr *variance = + new NumericTablePtr(result->get(low_order_moments::variance)); + NumericTablePtr *max = + new NumericTablePtr(result->get(low_order_moments::maximum)); + NumericTablePtr *min = + new NumericTablePtr(result->get(low_order_moments::minimum)); + + env->SetLongField(resultObj, meanNumericTableField, (jlong)mean); + env->SetLongField(resultObj, varianceNumericTableField, + (jlong)variance); + env->SetLongField(resultObj, maximumNumericTableField, (jlong)max); + env->SetLongField(resultObj, minimumNumericTableField, (jlong)min); + } +} + +/* + * Class: com_intel_oap_mllib_stat_CorrelationDALImpl + * Method: cCorrelationTrainDAL + * Signature: (JJDDIILorg/apache/spark/ml/stat/CorrelationResult;)J + */ +JNIEXPORT jlong JNICALL +Java_com_intel_oap_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL( + JNIEnv *env, jobject obj, jlong pNumTabData, jint executor_num, + jint executor_cores, jboolean use_gpu, jintArray gpu_idx_array, + jobject resultObj) { + + ccl::communicator &comm = getComm(); + size_t rankId = comm.rank(); + std::cout << " rankId : " << rankId << " ! " << std::endl; + + const size_t nBlocks = executor_num; + + NumericTablePtr pData = *((NumericTablePtr *)pNumTabData); + +#ifdef CPU_GPU_PROFILE + + if (use_gpu) { + + int n_gpu = env->GetArrayLength(gpu_idx_array); + cout << "oneDAL (native): use GPU kernels with " << n_gpu << " GPU(s)" + << endl; + + jint *gpu_indices = env->GetIntArrayElements(gpu_idx_array, 0); + + int size = comm.size(); + auto assigned_gpu = + getAssignedGPU(comm, size, rankId, gpu_indices, n_gpu); + + // Set SYCL context + cl::sycl::queue queue(assigned_gpu); + daal::services::SyclExecutionContext ctx(queue); + daal::services::Environment::getInstance()->setDefaultExecutionContext( + ctx); + + summarizer_compute(env, obj, rankId, comm, pData, nBlocks, resultObj); + env->ReleaseIntArrayElements(gpu_idx_array, gpu_indices, 0); + } else +#endif + { + // 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; + + summarizer_compute(env, obj, rankId, comm, pData, nBlocks, resultObj); + } + + return 0; +} diff --git a/mllib-dal/src/main/native/build-jni.sh b/mllib-dal/src/main/native/build-jni.sh index bd8ea9034..cb7d4c04f 100755 --- a/mllib-dal/src/main/native/build-jni.sh +++ b/mllib-dal/src/main/native/build-jni.sh @@ -36,4 +36,5 @@ javah -d $WORK_DIR/javah -classpath "$WORK_DIR/../../../target/classes:$DAAL_JAR com.intel.oap.mllib.recommendation.ALSDALImpl \ com.intel.oap.mllib.classification.NaiveBayesDALImpl \ com.intel.oap.mllib.regression.LinearRegressionDALImpl \ - com.intel.oap.mllib.stat.CorrelationDALImpl + com.intel.oap.mllib.stat.CorrelationDALImpl \ + com.intel.oap.mllib.stat.SummarizerDALImpl \ No newline at end of file diff --git a/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h new file mode 100644 index 000000000..60669f7cd --- /dev/null +++ b/mllib-dal/src/main/native/javah/com_intel_oap_mllib_stat_SummarizerDALImpl.h @@ -0,0 +1,21 @@ +/* DO NOT EDIT THIS FILE - it is machine generated */ +#include +/* Header for class com_intel_oap_mllib_stat_SummarizerDALImpl */ + +#ifndef _Included_com_intel_oap_mllib_stat_SummarizerDALImpl +#define _Included_com_intel_oap_mllib_stat_SummarizerDALImpl +#ifdef __cplusplus +extern "C" { +#endif +/* + * Class: com_intel_oap_mllib_stat_SummarizerDALImpl + * Method: cSummarizerTrainDAL + * Signature: (JIIZ[ILcom/intel/oap/mllib/stat/SummarizerResult;)J + */ +JNIEXPORT jlong JNICALL Java_com_intel_oap_mllib_stat_SummarizerDALImpl_cSummarizerTrainDAL + (JNIEnv *, jobject, jlong, jint, jint, jboolean, jintArray, jobject); + +#ifdef __cplusplus +} +#endif +#endif diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala index d3e3308e3..937740eb4 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala @@ -31,6 +31,10 @@ object Utils { return sc.getConf.getBoolean("spark.oap.mllib.enabled", true) } + def isOAPEnabled(sc: SparkContext): Boolean = { + return sc.getConf.getBoolean("spark.oap.mllib.enabled", true) + } + def getOneCCLIPPort(data: RDD[_]): String = { val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) val kvsIP = data.sparkContext.getConf.get("spark.oap.mllib.oneccl.kvs.ip", diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala new file mode 100644 index 000000000..525285079 --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala @@ -0,0 +1,121 @@ +/* + * 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 com.intel.oap.mllib.stat + +import com.intel.oap.mllib.{OneCCL, OneDAL} +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{Vector} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.stat.{MultivariateStatisticalDALSummary, MultivariateStatisticalSummary => Summary} +import org.apache.spark.rdd.RDD +import com.intel.oap.mllib.Utils.getOneCCLIPPort + + +class SummarizerDALImpl( + val executorNum: Int, + val executorCores: Int) + extends Serializable with Logging { + + def computeSummarizerMatrix(data: RDD[Vector]): Summary = { + val kvsIPPort = getOneCCLIPPort(data) + + val sparkContext = data.sparkContext + val useGPU = sparkContext.getConf.getBoolean("spark.oap.mllib.useGPU", false) + + val coalescedTables = OneDAL.rddVectorToMergedTables(data, executorNum) + + val results = coalescedTables.mapPartitionsWithIndex { (rank, table) => + + val gpuIndices = if (useGPU) { + val resources = TaskContext.get().resources() + resources("gpu").addresses.map(_.toInt) + } else { + null + } + + val tableArr = table.next() + OneCCL.init(executorNum, rank, kvsIPPort) + + val computeStartTime = System.nanoTime() + + val result = new SummarizerResult() + cSummarizerTrainDAL( + tableArr, + executorNum, + executorCores, + useGPU, + gpuIndices, + result + ) + + val computeEndTime = System.nanoTime() + + val durationCompute = (computeEndTime - computeStartTime).toDouble / 1E9 + + logInfo(s"SummarizerDAL compute took ${durationCompute} secs") + + val ret = if (OneCCL.isRoot()) { + + val convResultStartTime = System.nanoTime() + val meanMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.meanNumericTable)) + val varianceMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.varianceNumericTable)) + val maxMatrix= OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.maximumNumericTable)) + val minMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.minimumNumericTable)) + + val meanVector = OldVectors.dense(meanMatrix.toArray) + val varianceVector = OldVectors.dense(varianceMatrix.toArray) + val maxVector = OldVectors.dense(maxMatrix.toArray) + val minVector = OldVectors.dense(minMatrix.toArray) + + val convResultEndTime = System.nanoTime() + + val durationCovResult = (convResultEndTime - convResultStartTime).toDouble / 1E9 + + logInfo(s"SummarizerDAL result conversion took ${durationCovResult} secs") + + Iterator((meanVector, varianceVector, maxVector, minVector)) + } else { + Iterator.empty + } + + OneCCL.cleanup() + + ret + }.collect() + + // Make sure there is only one result from rank 0 + assert(results.length == 1) + + val meanVector = results(0)._1 + val varianceVector = results(0)._2 + val maxVector = results(0)._3 + val minVector = results(0)._4 + + val summary = new MultivariateStatisticalDALSummary(meanVector, varianceVector, maxVector, minVector) + + summary + } + + + @native private def cSummarizerTrainDAL(data: Long, + executor_num: Int, + executor_cores: Int, + useGPU: Boolean, + gpuIndices: Array[Int], + result: SummarizerResult): Long +} \ No newline at end of file diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala new file mode 100644 index 000000000..6136abe4d --- /dev/null +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala @@ -0,0 +1,43 @@ +/* + * 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 com.intel.oap.mllib.stat + +import org.apache.spark.{SPARK_VERSION, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset} + +import org.apache.spark.mllib.stat.spark320.{Statistics => SummarizerSpark320 } + +trait SummarizerShim extends Serializable with Logging { + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary + + } + +object SummarizerShim extends Logging { + def create(): SummarizerShim = { + logInfo(s"Loading Summarizer for Spark $SPARK_VERSION") + val summarizer = SPARK_VERSION match { + case "3.1.1" | "3.1.2" | "3.2.0" => new SummarizerSpark320() + case _ => throw new SparkException(s"Unsupported Spark version $SPARK_VERSION") + } + summarizer + } +} + diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala new file mode 100644 index 000000000..f323197ce --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -0,0 +1,65 @@ +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +class MultivariateStatisticalDALSummary ( + val meanVector: Vector, + val varianceVector: Vector, + val maxVector: Vector, + val minVector: Vector) + extends MultivariateStatisticalSummary with Serializable { + + /** + * Sample mean vector. + */ + override def mean: Vector = { + meanVector + } + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + override def variance: Vector = { + varianceVector + } + + /** + * Sample size. + */ + override def count: Long = 0 + + /** + * Sum of weights. + */ + override def weightSum: Double = 0.0 + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + override def numNonzeros: Vector = null + + /** + * Maximum value of each column. + */ + override def max: Vector = { + maxVector + } + + /** + * Minimum value of each column. + */ + override def min: Vector = { + minVector + } + + /** + * Euclidean magnitude of each column + */ + override def normL2: Vector = null + + /** + * L1 norm of each column + */ + override def normL1: Vector = null + +} \ No newline at end of file diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 000000000..a4443335c --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import com.intel.oap.mllib.stat.{CorrelationShim, SummarizerShim} +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.stat.correlation.Correlations + + + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +object Statistics { + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val shim = SummarizerShim.create() + shim.colStats(X) + } + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Since("1.1.0") + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + */ + @Since("1.1.0") + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + */ + @Since("1.1.0") + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * Java-friendly version of `corr()` + */ + @Since("1.4.1") + def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = + corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + +} diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala new file mode 100644 index 000000000..214d58d95 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.spark320 + +import com.intel.oap.mllib.Utils +import com.intel.oap.mllib.stat.{SummarizerDALImpl, SummarizerShim} +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import scala.annotation.varargs + + +/** + * API for statistical functions in MLlib. + */ +@Since("1.1.0") +class Statistics extends SummarizerShim{ + + /** + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[org.apache.spark.mllib.stat.MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Since("1.1.0") + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + val isPlatformSupported = Utils.checkClusterPlatformCompatibility( + X.sparkContext) + if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) + if (handlePersistence) { + X.persist(StorageLevel.MEMORY_AND_DISK) + } + val rdd = X.map { + v => v.asML + } + val executor_num = Utils.sparkExecutorNum(X.sparkContext) + val executor_cores = Utils.sparkExecutorCores() + val summary = new SummarizerDALImpl(executor_num, executor_cores) + .computeSummarizerMatrix(rdd) + if (handlePersistence) { + X.unpersist() + } + summary + } else { + new RowMatrix(X).computeColumnSummaryStatistics() + } + } +} From 0b24ebe32937079b78bfc341071818e6f68329df Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Fri, 26 Nov 2021 10:35:27 +0800 Subject: [PATCH 05/13] 1. fix CI error. 2. add unit test 3. modify README Signed-off-by: minmingzhu --- README.md | 3 + .../apache/spark/mllib/stat/Statistics.scala | 145 ++++++++- .../MultivariateOnlineSummarizerSuite.scala | 291 ++++++++++++++++++ mllib-dal/test.sh | 3 +- 4 files changed, 435 insertions(+), 7 deletions(-) create mode 100644 mllib-dal/src/test/scala/org/apache/spark/mllib.stat/MultivariateOnlineSummarizerSuite.scala diff --git a/README.md b/README.md index ecac590ee..8a820afa2 100644 --- a/README.md +++ b/README.md @@ -183,6 +183,8 @@ als | ALS example for Scala naive-bayes | Naive Bayes example for Scala linear-regression | Linear Regression example for Scala correlation | Correlation example for Scala +summarizer | summarizer example for Scala + ### Python Examples @@ -203,3 +205,4 @@ Naive Bayes | X | | Stable Linear Regression | X | | Experimental Ridge Regression | X | | Experimental Correlation | X | X | Experimental +Summarizer | X | X | Experimental diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index a4443335c..e9550f05b 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -17,15 +17,20 @@ package org.apache.spark.mllib.stat -import com.intel.oap.mllib.stat.{CorrelationShim, SummarizerShim} +import com.intel.oap.mllib.stat.{SummarizerShim} + +import scala.annotation.varargs + import org.apache.spark.annotation.Since -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.rdd.RDD +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.ml.stat._ import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.correlation.Correlations - - - +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, + KolmogorovSmirnovTestResult} +import org.apache.spark.rdd.RDD /** * API for statistical functions in MLlib. @@ -44,6 +49,22 @@ object Statistics { val shim = SummarizerShim.create() shim.colStats(X) } + + /** + * Computes required column-wise summary statistics for the input RDD[(Vector, Double)]. + * + * @param X an RDD containing vectors and weights for which column-wise summary statistics + * are to be computed. + * @return [[SummarizerBuffer]] object containing column-wise summary statistics. + */ + private[mllib] def colStats(X: RDD[(Vector, Double)], requested: Seq[String]) = { + X.treeAggregate(Summarizer.createSummarizerBuffer(requested: _*))( + seqOp = { case (c, (v, w)) => c.add(v.nonZeroIterator, v.size, w) }, + combOp = { case (c1, c2) => c1.merge(c2) }, + depth = 2 + ) + } + /** * Compute the Pearson correlation matrix for the input RDD of Vectors. * Columns with 0 covariance produce NaN entries in the correlation matrix. @@ -116,4 +137,116 @@ object Statistics { def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. + */ + @Since("1.1.0") + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Since("1.1.0") + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Since("1.1.0") + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } + + /** + * Java-friendly version of `chiSqTest()` + */ + @Since("1.5.0") + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) + + /** + * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a + * continuous distribution. By comparing the largest difference between the empirical cumulative + * distribution of the sample data and the theoretical distribution we can provide a test for the + * the null hypothesis that the sample data comes from that theoretical distribution. + * For more information on KS Test: + * @see + * Kolmogorov-Smirnov test (Wikipedia) + * + * @param data an `RDD[Double]` containing the sample of data to test + * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, cdf) + } + + /** + * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability + * distribution equality. Currently supports the normal distribution, taking as parameters + * the mean and standard deviation. + * (distName = "norm") + * @param data an `RDD[Double]` containing the sample of data to test + * @param distName a `String` name for a theoretical distribution + * @param params `Double*` specifying the parameters to be used for the theoretical distribution + * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test + * statistic, p-value, and null hypothesis. + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) + : KolmogorovSmirnovTestResult = { + KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) + } + + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ + @Since("1.5.0") + @varargs + def kolmogorovSmirnovTest( data: JavaDoubleRDD, + distName: String, + params: Double*): KolmogorovSmirnovTestResult = { + kolmogorovSmirnovTest(data.rdd.asInstanceOf[RDD[Double]], distName, params: _*) + } } diff --git a/mllib-dal/src/test/scala/org/apache/spark/mllib.stat/MultivariateOnlineSummarizerSuite.scala b/mllib-dal/src/test/scala/org/apache/spark/mllib.stat/MultivariateOnlineSummarizerSuite.scala new file mode 100644 index 000000000..c6466bc91 --- /dev/null +++ b/mllib-dal/src/test/scala/org/apache/spark/mllib.stat/MultivariateOnlineSummarizerSuite.scala @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.TestingUtils._ + +class MultivariateOnlineSummarizerSuite extends SparkFunSuite { + + test("basic error handing") { + val summarizer = new MultivariateOnlineSummarizer + + assert(summarizer.count === 0, "should be zero since nothing is added.") + + withClue("Getting numNonzeros from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.numNonzeros + } + } + + withClue("Getting variance from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.variance + } + } + + withClue("Getting mean from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.mean + } + } + + withClue("Getting max from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.max + } + } + + withClue("Getting min from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.min + } + } + + summarizer.add(Vectors.dense(-1.0, 2.0, 6.0)).add(Vectors.sparse(3, Seq((0, -2.0), (1, 6.0)))) + + withClue("Adding a new dense sample with different array size should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.add(Vectors.dense(3.0, 1.0)) + } + } + + withClue("Adding a new sparse sample with different array size should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.add(Vectors.sparse(5, Seq((0, -2.0), (1, 6.0)))) + } + } + + val summarizer2 = (new MultivariateOnlineSummarizer).add(Vectors.dense(1.0, -2.0, 0.0, 4.0)) + withClue("Merging a new summarizer with different dimensions should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.merge(summarizer2) + } + } + } + + test("dense vector input") { + // For column 2, the maximum will be 0.0, and it's not explicitly added since we ignore all + // the zeros; it's a case we need to test. For column 3, the minimum will be 0.0 which we + // need to test as well. + val summarizer = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(-1.0, 0.0, 6.0)) + .add(Vectors.dense(3.0, -3.0, 0.0)) + + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") + + assert(summarizer.count === 2) + } + + test("sparse vector input") { + val summarizer = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -1.0), (2, 6.0)))) + .add(Vectors.sparse(3, Seq((0, 3.0), (1, -3.0)))) + + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") + + assert(summarizer.count === 2) + } + + test("mixing dense and sparse vector input") { + val summarizer = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))) + .add(Vectors.dense(0.0, -1.0, -3.0)) + .add(Vectors.sparse(3, Seq((1, -5.1)))) + .add(Vectors.dense(3.8, 0.0, 1.9)) + .add(Vectors.dense(1.7, -0.6, 0.0)) + .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) + + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") + + assert(summarizer.count === 6) + } + + test("merging two summarizers") { + val summarizer1 = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))) + .add(Vectors.dense(0.0, -1.0, -3.0)) + + val summarizer2 = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((1, -5.1)))) + .add(Vectors.dense(3.8, 0.0, 1.9)) + .add(Vectors.dense(1.7, -0.6, 0.0)) + .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) + + val summarizer = summarizer1.merge(summarizer2) + + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") + + assert(summarizer.count === 6) + } + + test("merging summarizer with empty summarizer") { + // If one of two is non-empty, this should return the non-empty summarizer. + // If both of them are empty, then just return the empty summarizer. + val summarizer1 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(0.0, -1.0, -3.0)).merge(new MultivariateOnlineSummarizer) + assert(summarizer1.count === 1) + + val summarizer2 = (new MultivariateOnlineSummarizer) + .merge((new MultivariateOnlineSummarizer).add(Vectors.dense(0.0, -1.0, -3.0))) + assert(summarizer2.count === 1) + + val summarizer3 = (new MultivariateOnlineSummarizer).merge(new MultivariateOnlineSummarizer) + assert(summarizer3.count === 0) + + assert(summarizer1.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer2.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer1.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") + + assert(summarizer2.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") + + assert(summarizer1.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") + + assert(summarizer2.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") + + assert(summarizer1.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer2.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer1.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") + + assert(summarizer2.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") + } + + test("merging summarizer when one side has zero mean (SPARK-4355)") { + val s0 = new MultivariateOnlineSummarizer() + .add(Vectors.dense(2.0)) + .add(Vectors.dense(2.0)) + val s1 = new MultivariateOnlineSummarizer() + .add(Vectors.dense(1.0)) + .add(Vectors.dense(-1.0)) + s0.merge(s1) + assert(s0.mean(0) ~== 1.0 absTol 1e-14) + } + + test("merging summarizer with weighted samples") { + val summarizer = (new MultivariateOnlineSummarizer) + .add(instance = Vectors.sparse(3, Seq((0, -0.8), (1, 1.7))), weight = 0.1) + .add(Vectors.dense(0.0, -1.2, -1.7), 0.2).merge( + (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -0.7), (1, 0.01), (2, 1.3))), 0.15) + .add(Vectors.dense(-0.5, 0.3, -1.5), 0.05)) + + assert(summarizer.count === 4) + + // The following values are hand calculated using the formula: + // [[https://en.wikipedia.org/wiki/Weighted_arithmetic_mean#Reliability_weights]] + // which defines the reliability weight used for computing the unbiased estimation of variance + // for weighted instances. + assert(summarizer.mean ~== Vectors.dense(Array(-0.42, -0.107, -0.44)) + absTol 1E-10, "mean mismatch") + assert(summarizer.variance ~== Vectors.dense(Array(0.17657142857, 1.645115714, 2.42057142857)) + absTol 1E-8, "variance mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(Array(3.0, 4.0, 3.0)) + absTol 1E-10, "numNonzeros mismatch") + assert(summarizer.max ~== Vectors.dense(Array(0.0, 1.7, 1.3)) absTol 1E-10, "max mismatch") + assert(summarizer.min ~== Vectors.dense(Array(-0.8, -1.2, -1.7)) absTol 1E-10, "min mismatch") + assert(summarizer.normL2 ~== Vectors.dense(0.387298335, 0.762571308141, 0.9715966241192) + absTol 1E-8, "normL2 mismatch") + assert(summarizer.normL1 ~== Vectors.dense(0.21, 0.4265, 0.61) absTol 1E-10, "normL1 mismatch") + } + + test("test min/max with weighted samples (SPARK-16561)") { + val summarizer1 = new MultivariateOnlineSummarizer() + .add(Vectors.dense(10.0, -10.0), 1e10) + .add(Vectors.dense(0.0, 0.0), 1e-7) + + val summarizer2 = new MultivariateOnlineSummarizer() + summarizer2.add(Vectors.dense(10.0, -10.0), 1e10) + for (i <- 1 to 100) { + summarizer2.add(Vectors.dense(0.0, 0.0), 1e-7) + } + + val summarizer3 = new MultivariateOnlineSummarizer() + for (i <- 1 to 100) { + summarizer3.add(Vectors.dense(0.0, 0.0), 1e-7) + } + summarizer3.add(Vectors.dense(10.0, -10.0), 1e10) + + assert(summarizer1.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer1.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + assert(summarizer2.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer2.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + assert(summarizer3.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer3.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + } + + test ("test zero variance (SPARK-21818)") { + val summarizer1 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.7) + val summarizer2 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.4) + val summarizer3 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.5) + val summarizer4 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.4) + + val summarizer = summarizer1 + .merge(summarizer2) + .merge(summarizer3) + .merge(summarizer4) + + assert(summarizer.variance(0) >= 0.0) + } +} diff --git a/mllib-dal/test.sh b/mllib-dal/test.sh index 7147b59b3..6bb4f1652 100755 --- a/mllib-dal/test.sh +++ b/mllib-dal/test.sh @@ -52,7 +52,8 @@ suiteArray=( "recommendation.MLlibALSSuite" \ "classification.MLlibNaiveBayesSuite" \ "regression.MLlibLinearRegressionSuite" \ - "stat.MLlibCorrelationSuite" + "stat.MLlibCorrelationSuite" \ + "stat.MultivariateOnlineSummarizerSuite" ) MVN_NO_TRANSFER_PROGRESS= From 68d41dec000854e5fe78fc47dc3237642638ad68 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Fri, 26 Nov 2021 11:04:35 +0800 Subject: [PATCH 06/13] 1. change directory name Signed-off-by: minmingzhu --- .../stat}/MultivariateOnlineSummarizerSuite.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename mllib-dal/src/test/scala/org/apache/spark/{mllib.stat => mllib/stat}/MultivariateOnlineSummarizerSuite.scala (100%) diff --git a/mllib-dal/src/test/scala/org/apache/spark/mllib.stat/MultivariateOnlineSummarizerSuite.scala b/mllib-dal/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala similarity index 100% rename from mllib-dal/src/test/scala/org/apache/spark/mllib.stat/MultivariateOnlineSummarizerSuite.scala rename to mllib-dal/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala From bca35e15ccb60563a775228c23de42c7b6a6aef9 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Fri, 26 Nov 2021 14:11:25 +0800 Subject: [PATCH 07/13] 1. check codestyle 2. add numericTable1xNToVector function and instead of numericTableToOldMatrix in SummarizerDALImpl.scala Signed-off-by: minmingzhu --- examples/summarizer/pom.xml | 2 +- .../oap/mllib/stat/SummarizerResult.java | 13 +++++++ .../scala/com/intel/oap/mllib/OneDAL.scala | 15 ++++++++ .../oap/mllib/stat/SummarizerDALImpl.scala | 34 +++++++------------ .../intel/oap/mllib/stat/SummarizerShim.scala | 1 - .../mllib/stat/spark320/Statistics.scala | 2 +- 6 files changed, 43 insertions(+), 24 deletions(-) diff --git a/examples/summarizer/pom.xml b/examples/summarizer/pom.xml index 51aa36355..b9973c0e7 100644 --- a/examples/summarizer/pom.xml +++ b/examples/summarizer/pom.xml @@ -15,7 +15,7 @@ 1.3.0 2.12.10 2.12 - 3.1.1 + 3.2.0 diff --git a/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java b/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java index d70499768..002825440 100644 --- a/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java +++ b/mllib-dal/src/main/java/com/intel/oap/mllib/stat/SummarizerResult.java @@ -1,3 +1,16 @@ +/******************************************************************************* + * 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 com.intel.oap.mllib.stat; public class SummarizerResult { diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala index b5c429cb1..1f9cc7792 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala @@ -93,6 +93,21 @@ object OneDAL { Vectors.dense(arrayDouble) } + def numericTable1xNToVector(table: NumericTable): Vector = { + val numCols = table.getNumberOfColumns.toInt + + var dataDouble: DoubleBuffer = null + // returned DoubleBuffer is ByteByffer, need to copy as double array + dataDouble = table.getBlockOfRows(0, 1, dataDouble) + val arrayDouble = new Array[Double](numCols.toInt) + + dataDouble.get(arrayDouble) + + table.releaseBlockOfRows(0, 1, dataDouble) + + Vectors.dense(arrayDouble) + } + // Convert DAL numeric table to array of vectors def numericTableToVectors(table: NumericTable): Array[Vector] = { val numRows = table.getNumberOfRows.toInt diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala index 525285079..88d581bf2 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerDALImpl.scala @@ -25,10 +25,8 @@ import org.apache.spark.mllib.stat.{MultivariateStatisticalDALSummary, Multivari import org.apache.spark.rdd.RDD import com.intel.oap.mllib.Utils.getOneCCLIPPort - -class SummarizerDALImpl( - val executorNum: Int, - val executorCores: Int) +class SummarizerDALImpl(val executorNum: Int, + val executorCores: Int) extends Serializable with Logging { def computeSummarizerMatrix(data: RDD[Vector]): Summary = { @@ -72,15 +70,10 @@ class SummarizerDALImpl( val ret = if (OneCCL.isRoot()) { val convResultStartTime = System.nanoTime() - val meanMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.meanNumericTable)) - val varianceMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.varianceNumericTable)) - val maxMatrix= OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.maximumNumericTable)) - val minMatrix = OneDAL.numericTableToOldMatrix(OneDAL.makeNumericTable(result.minimumNumericTable)) - - val meanVector = OldVectors.dense(meanMatrix.toArray) - val varianceVector = OldVectors.dense(varianceMatrix.toArray) - val maxVector = OldVectors.dense(maxMatrix.toArray) - val minVector = OldVectors.dense(minMatrix.toArray) + val meanVector = OneDAL.numericTable1xNToVector(OneDAL.makeNumericTable(result.meanNumericTable)) + val varianceVector = OneDAL.numericTable1xNToVector(OneDAL.makeNumericTable(result.varianceNumericTable)) + val maxVector= OneDAL.numericTable1xNToVector(OneDAL.makeNumericTable(result.maximumNumericTable)) + val minVector = OneDAL.numericTable1xNToVector(OneDAL.makeNumericTable(result.minimumNumericTable)) val convResultEndTime = System.nanoTime() @@ -106,16 +99,15 @@ class SummarizerDALImpl( val maxVector = results(0)._3 val minVector = results(0)._4 - val summary = new MultivariateStatisticalDALSummary(meanVector, varianceVector, maxVector, minVector) + val summary = new MultivariateStatisticalDALSummary(OldVectors.fromML(meanVector), OldVectors.fromML(varianceVector), OldVectors.fromML(maxVector), OldVectors.fromML(minVector)) summary } - @native private def cSummarizerTrainDAL(data: Long, - executor_num: Int, - executor_cores: Int, - useGPU: Boolean, - gpuIndices: Array[Int], - result: SummarizerResult): Long -} \ No newline at end of file + executor_num: Int, + executor_cores: Int, + useGPU: Boolean, + gpuIndices: Array[Int], + result: SummarizerResult): Long +} diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala index 6136abe4d..b5648ca32 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/stat/SummarizerShim.scala @@ -40,4 +40,3 @@ object SummarizerShim extends Logging { summarizer } } - diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala index 214d58d95..cc3889dea 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala @@ -38,7 +38,7 @@ import scala.annotation.varargs * API for statistical functions in MLlib. */ @Since("1.1.0") -class Statistics extends SummarizerShim{ +class Statistics extends SummarizerShim { /** * Computes column-wise summary statistics for the input RDD[Vector]. From 71d37a3d1bbcb22632bfd575269703e11f8997ab Mon Sep 17 00:00:00 2001 From: minmingzhu <45281494+minmingzhu@users.noreply.github.com> Date: Fri, 26 Nov 2021 14:20:49 +0800 Subject: [PATCH 08/13] Update OneDAL.scala --- mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala index 1f9cc7792..0af24207c 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/OneDAL.scala @@ -100,7 +100,6 @@ object OneDAL { // returned DoubleBuffer is ByteByffer, need to copy as double array dataDouble = table.getBlockOfRows(0, 1, dataDouble) val arrayDouble = new Array[Double](numCols.toInt) - dataDouble.get(arrayDouble) table.releaseBlockOfRows(0, 1, dataDouble) From e061d7a2744e4295d0de470f039f84b2bc1074c9 Mon Sep 17 00:00:00 2001 From: minmingzhu <45281494+minmingzhu@users.noreply.github.com> Date: Fri, 26 Nov 2021 14:22:27 +0800 Subject: [PATCH 09/13] Update MultivariateStatisticalDALSummary.scala --- .../spark/mllib/stat/MultivariateStatisticalDALSummary.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala index f323197ce..de2012ba1 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalDALSummary.scala @@ -62,4 +62,4 @@ class MultivariateStatisticalDALSummary ( */ override def normL1: Vector = null -} \ No newline at end of file +} From 4f2c5b01231a6abe4980daac9e2180345506eed9 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Fri, 26 Nov 2021 15:26:09 +0800 Subject: [PATCH 10/13] 1. remove isOAPEnable function Signed-off-by: minmingzhu --- .../spark/examples/ml/SummaryStatisticsExample.scala | 8 ++++++-- mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala | 4 ---- .../org/apache/spark/mllib/stat/spark320/Statistics.scala | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala index a173e2782..877d8c524 100644 --- a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala +++ b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala @@ -32,9 +32,13 @@ import org.apache.spark.rdd.RDD object SummaryStatisticsExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("SummaryStatisticsExample") - val sc = new SparkContext(conf) + val spark = SparkSession + .builder + .appName("SummaryStatisticsExample") + .getOrCreate() + + val sc = spark.sparkContext // $example on$ val data = sc.parallelize( Seq( Vectors.dense(5.308206,9.869278,1.018934,4.292158,6.081011,6.585723,2.411094,4.767308,-3.256320,-6.029562), diff --git a/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala index 937740eb4..d3e3308e3 100644 --- a/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala +++ b/mllib-dal/src/main/scala/com/intel/oap/mllib/Utils.scala @@ -31,10 +31,6 @@ object Utils { return sc.getConf.getBoolean("spark.oap.mllib.enabled", true) } - def isOAPEnabled(sc: SparkContext): Boolean = { - return sc.getConf.getBoolean("spark.oap.mllib.enabled", true) - } - def getOneCCLIPPort(data: RDD[_]): String = { val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) val kvsIP = data.sparkContext.getConf.get("spark.oap.mllib.oneccl.kvs.ip", diff --git a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala index cc3889dea..c65b272d4 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/mllib/stat/spark320/Statistics.scala @@ -50,7 +50,7 @@ class Statistics extends SummarizerShim { def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { val isPlatformSupported = Utils.checkClusterPlatformCompatibility( X.sparkContext) - if (Utils.isOAPEnabled(X.sparkContext) && isPlatformSupported) { + if (Utils.isOAPEnabled() && isPlatformSupported) { val handlePersistence = (X.getStorageLevel == StorageLevel.NONE) if (handlePersistence) { X.persist(StorageLevel.MEMORY_AND_DISK) From c44e1b78a9061b2c486251820e2014da55438291 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Mon, 29 Nov 2021 10:11:25 +0800 Subject: [PATCH 11/13] 1. update README and SummaryStatisticsExample.scala Signed-off-by: minmingzhu --- README.md | 2 +- .../apache/spark/examples/ml/SummaryStatisticsExample.scala | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/README.md b/README.md index 8a820afa2..8dcc1c5c0 100644 --- a/README.md +++ b/README.md @@ -183,7 +183,7 @@ als | ALS example for Scala naive-bayes | Naive Bayes example for Scala linear-regression | Linear Regression example for Scala correlation | Correlation example for Scala -summarizer | summarizer example for Scala +summarizer | Summarizer example for Scala ### Python Examples diff --git a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala index 877d8c524..7c494a280 100644 --- a/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala +++ b/examples/summarizer/src/main/scala/org/apache/spark/examples/ml/SummaryStatisticsExample.scala @@ -58,11 +58,6 @@ object SummaryStatisticsExample { println(summary.variance) // column-wise variance println(summary.max) println(summary.min) - println(summary.count) - println(summary.normL1) - println(summary.normL2) - println(summary.weightSum) - println(summary.numNonzeros) // number of nonzeros in each column // $example off$ sc.stop() From eb6c01266ea88583eed1ef92e5acb4e63d56e6e3 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Mon, 29 Nov 2021 11:24:53 +0800 Subject: [PATCH 12/13] 1. modify permission Signed-off-by: minmingzhu --- examples/summarizer/build.sh | 0 examples/summarizer/run.sh | 0 2 files changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 examples/summarizer/build.sh mode change 100644 => 100755 examples/summarizer/run.sh diff --git a/examples/summarizer/build.sh b/examples/summarizer/build.sh old mode 100644 new mode 100755 diff --git a/examples/summarizer/run.sh b/examples/summarizer/run.sh old mode 100644 new mode 100755 From d971124fc7633c5d3118c64b51ed07725800a833 Mon Sep 17 00:00:00 2001 From: minmingzhu Date: Mon, 29 Nov 2021 11:58:39 +0800 Subject: [PATCH 13/13] 1. update run.sh Signed-off-by: minmingzhu --- examples/summarizer/run.sh | 4 ---- 1 file changed, 4 deletions(-) diff --git a/examples/summarizer/run.sh b/examples/summarizer/run.sh index 6a8bab7d3..acb8474d7 100755 --- a/examples/summarizer/run.sh +++ b/examples/summarizer/run.sh @@ -20,10 +20,6 @@ time $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ --conf "spark.shuffle.reduceLocality.enabled=false" \ --conf "spark.network.timeout=1200s" \ --conf "spark.task.maxFailures=1" \ - --conf "spark.eventLog.enabled=true" \ - --conf "spark.eventLog.dir=hdfs://beaver:9000/spark-history-server" \ - --conf "spark.history.fs.logDirectory=hdfs://beaver:9000/spark-history-server" \ - --conf "spark.driver.log.dfsDir=/user/spark/driverLogs" \ --jars $OAP_MLLIB_JAR \ --class $APP_CLASS \ $APP_JAR \