From 00ec07960807900115878eb485254f368d4e69bc Mon Sep 17 00:00:00 2001 From: Xiaochang Wu Date: Thu, 14 Jan 2021 13:40:14 +0800 Subject: [PATCH 1/9] Add missing build.sh --- mllib-dal/build.sh | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100755 mllib-dal/build.sh diff --git a/mllib-dal/build.sh b/mllib-dal/build.sh new file mode 100755 index 000000000..da1d8df75 --- /dev/null +++ b/mllib-dal/build.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +# Check envs for building +if [[ -z $JAVA_HOME ]]; then + echo $JAVA_HOME not defined! + 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 + +echo === Building Environments === +echo JAVA_HOME=$JAVA_HOME +echo DAALROOT=$DAALROOT +echo TBBROOT=$TBBROOT +echo CCL_ROOT=$CCL_ROOT +echo GCC Version: $(gcc -dumpversion) +echo ============================= + +mvn -DskipTests clean package From bf2df7e64b65cc81423abbb94115337fc3f6b096 Mon Sep 17 00:00:00 2001 From: Xiaochang Wu Date: Thu, 14 Jan 2021 13:41:18 +0800 Subject: [PATCH 2/9] Add ALS with oneDAL backend --- .gitignore | 1 + examples/als-hibench/build.sh | 3 + examples/als-hibench/pom.xml | 100 + examples/als-hibench/run-hibench-oap-mllib.sh | 73 + examples/als-hibench/run-hibench-vanilla.sh | 61 + .../hibench/sparkbench/ml/ALSExample.scala | 111 + examples/als-pyspark/als-pyspark.py | 67 + examples/als-pyspark/run.sh | 75 + .../ml/recommendation/ALSPartitionInfo.java | 6 + .../spark/ml/recommendation/ALSResult.java | 9 + .../org/apache/spark/ml/util/LibLoader.java | 3 + mllib-dal/src/main/native/ALSDALImpl.cpp | 683 ++++++ mllib-dal/src/main/native/ALSShuffle.cpp | 108 + mllib-dal/src/main/native/ALSShuffle.h | 17 + mllib-dal/src/main/native/Makefile | 4 +- mllib-dal/src/main/native/OneCCL.cpp | 7 + mllib-dal/src/main/native/OneDAL.cpp | 50 + mllib-dal/src/main/native/build-jni.sh | 4 +- ...pache_spark_ml_recommendation_ALSDALImpl.h | 29 + .../javah/org_apache_spark_ml_util_OneDAL__.h | 8 + mllib-dal/src/main/native/service.cpp | 4 + mllib-dal/src/main/native/service.h | 3 + .../apache/spark/ml/recommendation/ALS.scala | 1885 +++++++++++++++++ .../spark/ml/recommendation/ALSDALImpl.scala | 385 ++++ .../org/apache/spark/ml/util/OneDAL.scala | 3 + 25 files changed, 3696 insertions(+), 3 deletions(-) create mode 100755 examples/als-hibench/build.sh create mode 100644 examples/als-hibench/pom.xml create mode 100755 examples/als-hibench/run-hibench-oap-mllib.sh create mode 100755 examples/als-hibench/run-hibench-vanilla.sh create mode 100644 examples/als-hibench/src/main/scala/com/intel/hibench/sparkbench/ml/ALSExample.scala create mode 100644 examples/als-pyspark/als-pyspark.py create mode 100755 examples/als-pyspark/run.sh create mode 100644 mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java create mode 100644 mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java create mode 100644 mllib-dal/src/main/native/ALSDALImpl.cpp create mode 100644 mllib-dal/src/main/native/ALSShuffle.cpp create mode 100644 mllib-dal/src/main/native/ALSShuffle.h create mode 100644 mllib-dal/src/main/native/javah/org_apache_spark_ml_recommendation_ALSDALImpl.h create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala create mode 100644 mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala diff --git a/.gitignore b/.gitignore index 21a5d40c2..1d621bdd4 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,7 @@ *.o *.log .vscode +*.iml target/ .idea/ .idea_modules/ diff --git a/examples/als-hibench/build.sh b/examples/als-hibench/build.sh new file mode 100755 index 000000000..8cbc692be --- /dev/null +++ b/examples/als-hibench/build.sh @@ -0,0 +1,3 @@ +#!/usr/bin/env bash + +mvn clean package \ No newline at end of file diff --git a/examples/als-hibench/pom.xml b/examples/als-hibench/pom.xml new file mode 100644 index 000000000..68e02c256 --- /dev/null +++ b/examples/als-hibench/pom.xml @@ -0,0 +1,100 @@ + + 4.0.0 + + com.intel.oap + oap-mllib-examples + 0.9.0-with-spark-3.0.0 + jar + + ALSHiBenchExample + https://github.com/Intel-bigdata/OAP + + + UTF-8 + 2.12.10 + 2.12 + 3.0.0 + + + + + + org.scala-lang + scala-library + 2.12.10 + + + + com.github.scopt + scopt_2.12 + 3.7.0 + + + + + + + + + + + org.apache.spark + spark-sql_2.12 + ${spark.version} + provided + + + + org.apache.spark + spark-mllib_2.12 + ${spark.version} + provided + + + + + + + + org.scala-tools + maven-scala-plugin + 2.15.2 + + + + compile + testCompile + + + + + ${scala.version} + + -target:jvm-1.8 + + + + + maven-assembly-plugin + 3.0.0 + + false + + jar-with-dependencies + + + + + assembly + package + + single + + + + + + + + diff --git a/examples/als-hibench/run-hibench-oap-mllib.sh b/examples/als-hibench/run-hibench-oap-mllib.sh new file mode 100755 index 000000000..050b80558 --- /dev/null +++ b/examples/als-hibench/run-hibench-oap-mllib.sh @@ -0,0 +1,73 @@ +#!/usr/bin/env bash + +export HDFS_ROOT=hdfs://sr591:8020 +export OAP_MLLIB_ROOT=/home/xiaochang/Works/OAP-xwu99-als/oap-mllib + +SPARK_MASTER=yarn +SPARK_DRIVER_MEMORY=16G +SPARK_NUM_EXECUTORS=6 +SPARK_EXECUTOR_CORES=28 +SPARK_EXECUTOR_MEMORY_OVERHEAD=25G +SPARK_EXECUTOR_MEMORY=100G + +SPARK_DEFAULT_PARALLELISM=$(expr $SPARK_NUM_EXECUTORS '*' $SPARK_EXECUTOR_CORES '*' 2) +#SPARK_DEFAULT_PARALLELISM=$(expr $SPARK_NUM_EXECUTORS '*' $SPARK_EXECUTOR_CORES) + +# ======================================================= # + +# for log suffix +SUFFIX=$( basename -s .sh "${BASH_SOURCE[0]}" ) + +# Check envs +if [[ -z $SPARK_HOME ]]; then + echo SPARK_HOME not defined! + exit 1 +fi + +if [[ -z $HADOOP_HOME ]]; then + echo HADOOP_HOME not defined! + exit 1 +fi + +export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop + +# Target jar built +OAP_MLLIB_JAR_NAME=oap-mllib-0.9.0-with-spark-3.0.0.jar +OAP_MLLIB_JAR=$OAP_MLLIB_ROOT/mllib-dal/target/$OAP_MLLIB_JAR_NAME + +# Use absolute path +SPARK_DRIVER_CLASSPATH=$OAP_MLLIB_JAR +# Use relative path +SPARK_EXECUTOR_CLASSPATH=./$OAP_MLLIB_JAR_NAME + +APP_JAR=target/oap-mllib-examples-0.9.0-with-spark-3.0.0.jar +APP_CLASS=com.intel.hibench.sparkbench.ml.ALSExample + +HDFS_INPUT=hdfs://sr591:8020/HiBench/ALS/Input +RANK=10 +NUM_ITERATIONS=1 +LAMBDA=0.1 +IMPLICIT=true + +/usr/bin/time -p $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --conf "spark.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.executor.memoryOverhead=$SPARK_EXECUTOR_MEMORY_OVERHEAD" \ + --conf "spark.network.timeout=1200s" \ + --conf "spark.task.maxFailures=1" \ + --jars $OAP_MLLIB_JAR \ + --class $APP_CLASS \ + $APP_JAR \ + --rank $RANK --numIterations $NUM_ITERATIONS --implicitPrefs $IMPLICIT --lambda $LAMBDA \ + --numProductBlocks $SPARK_DEFAULT_PARALLELISM --numUserBlocks $SPARK_DEFAULT_PARALLELISM \ + $HDFS_INPUT \ + 2>&1 | tee ALS-$SUFFIX-$(date +%m%d_%H_%M_%S).log + diff --git a/examples/als-hibench/run-hibench-vanilla.sh b/examples/als-hibench/run-hibench-vanilla.sh new file mode 100755 index 000000000..6cb6b3ae7 --- /dev/null +++ b/examples/als-hibench/run-hibench-vanilla.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash + +export HDFS_ROOT=hdfs://sr591:8020 + +SPARK_MASTER=yarn +SPARK_DRIVER_MEMORY=16G +SPARK_NUM_EXECUTORS=6 +SPARK_EXECUTOR_CORES=28 +SPARK_EXECUTOR_MEMORY_OVERHEAD=25G +SPARK_EXECUTOR_MEMORY=100G + +SPARK_DEFAULT_PARALLELISM=$(expr $SPARK_NUM_EXECUTORS '*' $SPARK_EXECUTOR_CORES '*' 2) + +# ======================================================= # + +# for log suffix +SUFFIX=$( basename -s .sh "${BASH_SOURCE[0]}" ) + +# Check envs +if [[ -z $SPARK_HOME ]]; then + echo SPARK_HOME not defined! + exit 1 +fi + +if [[ -z $HADOOP_HOME ]]; then + echo HADOOP_HOME not defined! + exit 1 +fi + +export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop + +APP_JAR=target/oap-mllib-examples-0.9.0-with-spark-3.0.0.jar +APP_CLASS=com.intel.hibench.sparkbench.ml.ALSExample + +HDFS_INPUT=hdfs://sr591:8020/HiBench/ALS/Input +RANK=10 +NUM_ITERATIONS=1 +LAMBDA=0.1 +IMPLICIT=true + +/usr/bin/time -p $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --conf "spark.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.executor.memoryOverhead=$SPARK_EXECUTOR_MEMORY_OVERHEAD" \ + --conf "spark.network.timeout=1200s" \ + --conf "spark.task.maxFailures=1" \ + --class $APP_CLASS \ + $APP_JAR \ + --rank $RANK --numIterations $NUM_ITERATIONS --implicitPrefs $IMPLICIT --lambda $LAMBDA \ + --numProductBlocks $SPARK_DEFAULT_PARALLELISM --numUserBlocks $SPARK_DEFAULT_PARALLELISM \ + $HDFS_INPUT \ + 2>&1 | tee ALS-$SUFFIX-$(date +%m%d_%H_%M_%S).log + diff --git a/examples/als-hibench/src/main/scala/com/intel/hibench/sparkbench/ml/ALSExample.scala b/examples/als-hibench/src/main/scala/com/intel/hibench/sparkbench/ml/ALSExample.scala new file mode 100644 index 000000000..5a29bcc80 --- /dev/null +++ b/examples/als-hibench/src/main/scala/com/intel/hibench/sparkbench/ml/ALSExample.scala @@ -0,0 +1,111 @@ +/* + * 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 com.intel.hibench.sparkbench.ml + +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.recommendation.ALS +import org.apache.spark.ml.recommendation.ALS.Rating +import org.apache.spark.sql.SparkSession +import scopt.OptionParser + +object ALSExample { + + case class Params( + dataPath: String = null, + numIterations: Int = 10, + lambda: Double = 0.1, + rank: Int = 10, + numUserBlocks: Int = 10, + numItemBlocks: Int = 10, + implicitPrefs: Boolean = false) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("ALS") { + head("ALS: an example app for ALS on User-Item data.") + opt[Int]("rank") + .text(s"rank, default: ${defaultParams.rank}") + .action((x, c) => c.copy(rank = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default: ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("lambda") + .text(s"regularization parameter, default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + opt[Int]("numUserBlocks") + .text(s"number of user blocks, default: ${defaultParams.numUserBlocks}") + .action((x, c) => c.copy(numUserBlocks = x)) + opt[Int]("numProductBlocks") + .text(s"number of product blocks, default: ${defaultParams.numItemBlocks}") + .action((x, c) => c.copy(numItemBlocks = x)) + opt[Boolean]("implicitPrefs") + .text("implicit preference, default: ${defaultParams.implicitPrefs}") + .action((x, c) => c.copy(implicitPrefs = x)) + arg[String]("") + .required() + .text("Input paths to a User-Product dataset of ratings") + .action((x, c) => c.copy(dataPath = x)) + } + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) + } + } + + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"ALS with $params") + .getOrCreate() + val sc = spark.sparkContext + + import spark.implicits._ + + val ratings = sc.objectFile[Rating[Int]](params.dataPath).toDF() + + val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2), 1L) + + // Build the recommendation model using ALS on the training data + val als = new ALS() + .setRank(params.rank) + .setMaxIter(params.numIterations) + .setRegParam(params.lambda) + .setImplicitPrefs(params.implicitPrefs) + .setNumUserBlocks(params.numUserBlocks) + .setNumItemBlocks(params.numItemBlocks) + .setUserCol("user") + .setItemCol("item") + .setRatingCol("rating") + val model = als.fit(training) + + // Evaluate the model by computing the RMSE on the test data + // Note we set cold start strategy to 'drop' to ensure we don't get NaN evaluation metrics + model.setColdStartStrategy("drop") + val predictions = model.transform(test) + + val evaluator = new RegressionEvaluator() + .setMetricName("rmse") + .setLabelCol("rating") + .setPredictionCol("prediction") + val rmse = evaluator.evaluate(predictions) + println(s"Root-mean-square error = $rmse") + + spark.stop() + } +} diff --git a/examples/als-pyspark/als-pyspark.py b/examples/als-pyspark/als-pyspark.py new file mode 100644 index 000000000..8847ca2b9 --- /dev/null +++ b/examples/als-pyspark/als-pyspark.py @@ -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. +# + +from __future__ import print_function + +import sys +if sys.version >= '3': + long = int + +from pyspark.sql import SparkSession + +# $example on$ +from pyspark.ml.evaluation import RegressionEvaluator +from pyspark.ml.recommendation import ALS +from pyspark.sql import Row +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("ALSExample")\ + .getOrCreate() + + if (len(sys.argv) != 2) : + print("Require data file path as input parameter") + sys.exit(1) + + # $example on$ + lines = spark.read.text(sys.argv[1]).rdd + parts = lines.map(lambda row: row.value.split("::")) + ratingsRDD = parts.map(lambda p: Row(userId=int(p[0]), movieId=int(p[1]), + rating=float(p[2]))) + ratings = spark.createDataFrame(ratingsRDD) + # (training, test) = ratings.randomSplit([0.8, 0.2]) + + # Build the recommendation model using ALS on the training data + # Note we set cold start strategy to 'drop' to ensure we don't get NaN evaluation metrics + als = ALS(rank=10, maxIter=5, regParam=0.01, implicitPrefs=True, alpha=40.0, + userCol="userId", itemCol="movieId", ratingCol="rating", + coldStartStrategy="drop") + print("\nALS training with implicitPrefs={}, rank={}, maxIter={}, regParam={}, alpha={}, seed={}\n".format( + als.getImplicitPrefs(), als.getRank(), als.getMaxIter(), als.getRegParam(), als.getAlpha(), als.getSeed() + )) + model = als.fit(ratings) + + # Evaluate the model by computing the RMSE on the test data + # predictions = model.transform(test) + # evaluator = RegressionEvaluator(metricName="rmse", labelCol="rating", + # predictionCol="prediction") + # rmse = evaluator.evaluate(predictions) + # print("Root-mean-square error = " + str(rmse)) + + spark.stop() diff --git a/examples/als-pyspark/run.sh b/examples/als-pyspark/run.sh new file mode 100755 index 000000000..b3ba1b6d2 --- /dev/null +++ b/examples/als-pyspark/run.sh @@ -0,0 +1,75 @@ +#!/usr/bin/env bash + +# == User to customize the following environments ======= # + +# Set user Spark and Hadoop home directory +#export SPARK_HOME=/path/to/your/spark/home +#export HADOOP_HOME=/path/to/your/hadoop/home +# Set user HDFS Root +export HDFS_ROOT=hdfs://sr549:8020 +# Set user Intel MLlib Root directory +export OAP_MLLIB_ROOT=/home/xiaochang/Works/OAP-xwu99-als/oap-mllib +# Set IP and Port for oneCCL KVS, you can select any one of the worker nodes and set CCL_KVS_IP_PORT to its IP and Port +# IP can be got with `hostname -I`, if multiple IPs are returned, the first IP should be used. Port can be any available port. +# For example, if one of the worker IP is 192.168.0.1 and an available port is 51234. +# CCL_KVS_IP_PORT can be set in the format of 192.168.0.1_51234 +# Incorrectly setting this value will result in hanging when oneCCL initialize +export CCL_KVS_IP_PORT=10.0.2.149_51234 + +# Data file is from Spark Examples (data/mllib/sample_kmeans_data.txt), the data file should be copied to HDFS +DATA_FILE=data/onedal_als_csr_ratings.txt + +# == User to customize Spark executor cores and memory == # + +# User should check the requested resources are acturally allocated by cluster manager or Intel MLlib will behave incorrectly +SPARK_MASTER=yarn +SPARK_DRIVER_MEMORY=1G +SPARK_NUM_EXECUTORS=2 +SPARK_EXECUTOR_CORES=1 +SPARK_EXECUTOR_MEMORY=1G + +SPARK_DEFAULT_PARALLELISM=$(expr $SPARK_NUM_EXECUTORS '*' $SPARK_EXECUTOR_CORES '*' 2) + +# ======================================================= # + +# Check env +if [[ -z $SPARK_HOME ]]; then + echo SPARK_HOME not defined! + exit 1 +fi + +if [[ -z $HADOOP_HOME ]]; then + echo HADOOP_HOME not defined! + exit 1 +fi + +export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop + +# Target jar built +OAP_MLLIB_JAR_NAME=oap-mllib-0.9.0-with-spark-3.0.0.jar +OAP_MLLIB_JAR=$OAP_MLLIB_ROOT/mllib-dal/target/$OAP_MLLIB_JAR_NAME + +# Use absolute path +SPARK_DRIVER_CLASSPATH=$OAP_MLLIB_JAR +# Use relative path +SPARK_EXECUTOR_CLASSPATH=./$OAP_MLLIB_JAR_NAME + +APP_PY=als-pyspark.py + +/usr/bin/time -p $SPARK_HOME/bin/spark-submit --master $SPARK_MASTER -v \ + --num-executors $SPARK_NUM_EXECUTORS \ + --driver-memory $SPARK_DRIVER_MEMORY \ + --executor-cores $SPARK_EXECUTOR_CORES \ + --executor-memory $SPARK_EXECUTOR_MEMORY \ + --conf "spark.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.executorEnv.CCL_KVS_IP_PORT=$CCL_KVS_IP_PORT" \ + --conf "spark.shuffle.reduceLocality.enabled=false" \ + --conf "spark.network.timeout=1200s" \ + --conf "spark.task.maxFailures=1" \ + --jars $OAP_MLLIB_JAR \ + $APP_PY $DATA_FILE \ + 2>&1 | tee ALS-$(date +%m%d_%H_%M_%S).log diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java b/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java new file mode 100644 index 000000000..5f2561772 --- /dev/null +++ b/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSPartitionInfo.java @@ -0,0 +1,6 @@ +package org.apache.spark.ml.recommendation; + +public class ALSPartitionInfo { + public int ratingsNum; + public int csrRowNum; +} diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java b/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java new file mode 100644 index 000000000..67173a0c5 --- /dev/null +++ b/mllib-dal/src/main/java/org/apache/spark/ml/recommendation/ALSResult.java @@ -0,0 +1,9 @@ +package org.apache.spark.ml.recommendation; + +public class ALSResult { + public long rankId = -1; + public long cUsersFactorsNumTab; + public long cItemsFactorsNumTab; + public long cUserOffset; + public long cItemOffset; +} diff --git a/mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java b/mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java index 5b51451ae..04c574d88 100644 --- a/mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java +++ b/mllib-dal/src/main/java/org/apache/spark/ml/util/LibLoader.java @@ -60,6 +60,9 @@ public static synchronized void loadLibCCL() throws IOException { loadFromJar(subDir, "libfabric.so.1"); loadFromJar(subDir, "libsockets-fi.so"); loadFromJar(subDir, "libccl_atl_ofi.so"); + + //loadFromJar(subDir, "libtbbmalloc.so.2"); + //loadFromJar(subDir, "libtbb.so.2"); } /** diff --git a/mllib-dal/src/main/native/ALSDALImpl.cpp b/mllib-dal/src/main/native/ALSDALImpl.cpp new file mode 100644 index 000000000..86dca0d4c --- /dev/null +++ b/mllib-dal/src/main/native/ALSDALImpl.cpp @@ -0,0 +1,683 @@ +#include +#include +#include + +#include +#include + +#include "ALSShuffle.h" +#include "org_apache_spark_ml_recommendation_ALSDALImpl.h" +#include "service.h" + +using namespace std; +using namespace daal; +using namespace daal::algorithms; +using namespace daal::algorithms::implicit_als; + +const int ccl_root = 0; + +typedef float algorithmFPType; /* Algorithm floating-point type */ + +NumericTablePtr userOffset; +NumericTablePtr itemOffset; + +// KeyValueDataCollectionPtr userOffsetsOnMaster; +// KeyValueDataCollectionPtr itemOffsetsOnMaster; + +CSRNumericTablePtr dataTable; +CSRNumericTablePtr transposedDataTable; + +KeyValueDataCollectionPtr userStep3LocalInput; +KeyValueDataCollectionPtr itemStep3LocalInput; + +training::DistributedPartialResultStep4Ptr itemsPartialResultLocal; +training::DistributedPartialResultStep4Ptr usersPartialResultLocal; +std::vector itemsPartialResultsMaster; +std::vector usersPartialResultsMaster; + +template +void gather(size_t rankId, size_t nBlocks, const ByteBuffer& nodeResults, T* result) { + size_t perNodeArchLengthMaster[nBlocks]; + size_t perNodeArchLength = nodeResults.size(); + ByteBuffer serializedData; + ccl_request_t request; + + size_t recv_counts[nBlocks]; + for (size_t i = 0; i < nBlocks; i++) recv_counts[i] = sizeof(size_t); + + // MPI_Gather(&perNodeArchLength, sizeof(int), MPI_CHAR, perNodeArchLengthMaster, + // sizeof(int), MPI_CHAR, ccl_root, MPI_COMM_WORLD); + ccl_allgatherv(&perNodeArchLength, sizeof(size_t), perNodeArchLengthMaster, recv_counts, + ccl_dtype_char, NULL, NULL, NULL, &request); + ccl_wait(request); + + // should resize for all ranks for ccl_allgatherv + size_t memoryBuf = 0; + for (size_t i = 0; i < nBlocks; i++) { + memoryBuf += perNodeArchLengthMaster[i]; + } + serializedData.resize(memoryBuf); + + std::vector displs(nBlocks); + if (rankId == ccl_root) { + size_t shift = 0; + for (size_t i = 0; i < nBlocks; i++) { + displs[i] = shift; + shift += perNodeArchLengthMaster[i]; + } + } + + /* Transfer partial results to step 2 on the root node */ + // MPI_Gatherv(&nodeResults[0], perNodeArchLength, MPI_CHAR, &serializedData[0], + // perNodeArchLengthMaster, displs, MPI_CHAR, ccl_root, + // MPI_COMM_WORLD); + ccl_allgatherv(&nodeResults[0], perNodeArchLength, &serializedData[0], + perNodeArchLengthMaster, ccl_dtype_char, NULL, NULL, NULL, &request); + ccl_wait(request); + + if (rankId == ccl_root) { + for (size_t i = 0; i < nBlocks; i++) { + /* Deserialize partial results from step 1 */ + result[i] = result[i]->cast(deserializeDAALObject(&serializedData[0] + displs[i], + perNodeArchLengthMaster[i])); + } + } +} + +// void gatherUsers(const ByteBuffer & nodeResults, int nBlocks) +// { +// size_t perNodeArchLengthMaster[nBlocks]; +// size_t perNodeArchLength = nodeResults.size(); +// ByteBuffer serializedData; +// size_t recv_counts[nBlocks]; +// for (int i = 0; i < nBlocks; i++) { +// recv_counts[i] = sizeof(size_t); +// } + +// ccl_request_t request; +// // MPI_Allgather(&perNodeArchLength, sizeof(int), MPI_CHAR, +// perNodeArchLengthMaster, sizeof(int), MPI_CHAR, MPI_COMM_WORLD); +// ccl_allgatherv(&perNodeArchLength, sizeof(size_t), perNodeArchLengthMaster, +// recv_counts, ccl_dtype_char, NULL, NULL, NULL, &request); ccl_wait(request); + +// size_t memoryBuf = 0; +// for (int i = 0; i < nBlocks; i++) +// { +// memoryBuf += perNodeArchLengthMaster[i]; +// } +// serializedData.resize(memoryBuf); + +// size_t shift = 0; +// std::vector displs(nBlocks); +// for (int i = 0; i < nBlocks; i++) +// { +// displs[i] = shift; +// shift += perNodeArchLengthMaster[i]; +// } + +// /* Transfer partial results to step 2 on the root node */ +// // MPI_Allgatherv(&nodeResults[0], perNodeArchLength, MPI_CHAR, &serializedData[0], +// perNodeArchLengthMaster, displs, MPI_CHAR, MPI_COMM_WORLD); +// ccl_allgatherv(&nodeResults[0], perNodeArchLength, &serializedData[0], +// perNodeArchLengthMaster, ccl_dtype_char, NULL, NULL, NULL, &request); +// ccl_wait(request); + +// usersPartialResultsMaster.resize(nBlocks); +// for (int i = 0; i < nBlocks; i++) +// { +// /* Deserialize partial results from step 4 */ +// usersPartialResultsMaster[i] = +// training::DistributedPartialResultStep4::cast(deserializeDAALObject(&serializedData[0] +// + displs[i], perNodeArchLengthMaster[i])); +// } +// } + +// void gatherItems(const ByteBuffer & nodeResults, size_t nBlocks) +// { +// size_t perNodeArchLengthMaster[nBlocks]; +// size_t perNodeArchLength = nodeResults.size(); +// ByteBuffer serializedData; +// size_t recv_counts[nBlocks]; +// for (size_t i = 0; i < nBlocks; i++) { +// recv_counts[i] = sizeof(size_t); +// } + +// ccl_request_t request; +// // MPI_Allgather(&perNodeArchLength, sizeof(int), MPI_CHAR, +// perNodeArchLengthMaster, sizeof(int), MPI_CHAR, MPI_COMM_WORLD); +// ccl_allgatherv(&perNodeArchLength, sizeof(size_t), perNodeArchLengthMaster, +// recv_counts, ccl_dtype_char, NULL, NULL, NULL, &request); ccl_wait(request); + +// size_t memoryBuf = 0; +// for (size_t i = 0; i < nBlocks; i++) +// { +// memoryBuf += perNodeArchLengthMaster[i]; +// } +// serializedData.resize(memoryBuf); + +// size_t shift = 0; +// std::vector displs(nBlocks); +// for (size_t i = 0; i < nBlocks; i++) +// { +// displs[i] = shift; +// shift += perNodeArchLengthMaster[i]; +// } + +// /* Transfer partial results to step 2 on the root node */ +// // MPI_Allgatherv(&nodeResults[0], perNodeArchLength, MPI_CHAR, &serializedData[0], +// perNodeArchLengthMaster, displs, MPI_CHAR, MPI_COMM_WORLD); +// ccl_allgatherv(&nodeResults[0], perNodeArchLength, &serializedData[0], +// perNodeArchLengthMaster, ccl_dtype_char, NULL, NULL, NULL, &request); +// ccl_wait(request); + +// itemsPartialResultsMaster.resize(nBlocks); +// for (size_t i = 0; i < nBlocks; i++) +// { +// /* Deserialize partial results from step 4 */ +// itemsPartialResultsMaster[i] = +// training::DistributedPartialResultStep4::cast(deserializeDAALObject(&serializedData[0] +// + displs[i], perNodeArchLengthMaster[i])); +// } +// } + +template +void all2all(ByteBuffer* nodeResults, size_t nBlocks, KeyValueDataCollectionPtr result) { + size_t memoryBuf = 0; + size_t shift = 0; + size_t perNodeArchLengths[nBlocks]; + size_t perNodeArchLengthsRecv[nBlocks]; + std::vector sdispls(nBlocks); + ByteBuffer serializedSendData; + ByteBuffer serializedRecvData; + + for (size_t i = 0; i < nBlocks; i++) { + perNodeArchLengths[i] = nodeResults[i].size(); + memoryBuf += perNodeArchLengths[i]; + sdispls[i] = shift; + shift += perNodeArchLengths[i]; + } + serializedSendData.resize(memoryBuf); + + /* memcpy to avoid double compute */ + memoryBuf = 0; + for (size_t i = 0; i < nBlocks; i++) { + for (size_t j = 0; j < perNodeArchLengths[i]; j++) + serializedSendData[memoryBuf + j] = nodeResults[i][j]; + memoryBuf += perNodeArchLengths[i]; + } + + ccl_request_t request; + // MPI_Alltoall(perNodeArchLengths, sizeof(int), MPI_CHAR, perNodeArchLengthsRecv, + // sizeof(int), MPI_CHAR, MPI_COMM_WORLD); + ccl_alltoall(perNodeArchLengths, perNodeArchLengthsRecv, sizeof(size_t), ccl_dtype_char, + NULL, NULL, NULL, &request); + ccl_wait(request); + + memoryBuf = 0; + shift = 0; + std::vector rdispls(nBlocks); + for (size_t i = 0; i < nBlocks; i++) { + memoryBuf += perNodeArchLengthsRecv[i]; + rdispls[i] = shift; + shift += perNodeArchLengthsRecv[i]; + } + + serializedRecvData.resize(memoryBuf); + + /* Transfer partial results to step 2 on the root node */ + // MPI_Alltoallv(&serializedSendData[0], perNodeArchLengths, sdispls, MPI_CHAR, + // &serializedRecvData[0], perNodeArchLengthsRecv, rdispls, MPI_CHAR, + // MPI_COMM_WORLD); + ccl_alltoallv(&serializedSendData[0], perNodeArchLengths, &serializedRecvData[0], + perNodeArchLengthsRecv, ccl_dtype_char, NULL, NULL, NULL, &request); + ccl_wait(request); + + for (size_t i = 0; i < nBlocks; i++) { + (*result)[i] = T::cast(deserializeDAALObject(&serializedRecvData[rdispls[i]], + perNodeArchLengthsRecv[i])); + } +} + +KeyValueDataCollectionPtr initializeStep1Local(size_t rankId, size_t partitionId, + size_t nBlocks, size_t nUsers, + size_t nFactors) { + int usersPartition[1] = {(int)nBlocks}; + + /* Create an algorithm object to initialize the implicit ALS model with the default + * method */ + training::init::Distributed + initAlgorithm; + initAlgorithm.parameter.fullNUsers = nUsers; + initAlgorithm.parameter.nFactors = nFactors; + initAlgorithm.parameter.seed += rankId; + initAlgorithm.parameter.partition.reset( + new HomogenNumericTable((int*)usersPartition, 1, 1)); + /* Pass a training data set and dependent values to the algorithm */ + initAlgorithm.input.set(training::init::data, dataTable); + + /* Initialize the implicit ALS model */ + initAlgorithm.compute(); + + training::init::PartialResultPtr partialResult = initAlgorithm.getPartialResult(); + itemStep3LocalInput = partialResult->get(training::init::outputOfInitForComputeStep3); + userOffset = partialResult->get(training::init::offsets, (size_t)rankId); + // if (rankId == ccl_root) + // { + // userOffsetsOnMaster = partialResult->get(training::init::offsets); + // } + PartialModelPtr partialModelLocal = partialResult->get(training::init::partialModel); + + itemsPartialResultLocal.reset(new training::DistributedPartialResultStep4()); + itemsPartialResultLocal->set(training::outputOfStep4ForStep1, partialModelLocal); + + return partialResult->get(training::init::outputOfStep1ForStep2); +} + +void initializeStep2Local(size_t rankId, size_t partitionId, + const KeyValueDataCollectionPtr& initStep2LocalInput) { + /* Create an algorithm object to perform the second step of the implicit ALS + * initialization algorithm */ + training::init::Distributed + initAlgorithm; + + initAlgorithm.input.set(training::init::inputOfStep2FromStep1, initStep2LocalInput); + + /* Compute partial results of the second step on local nodes */ + initAlgorithm.compute(); + + training::init::DistributedPartialResultStep2Ptr partialResult = + initAlgorithm.getPartialResult(); + transposedDataTable = + CSRNumericTable::cast(partialResult->get(training::init::transposedData)); + userStep3LocalInput = partialResult->get(training::init::outputOfInitForComputeStep3); + itemOffset = partialResult->get(training::init::offsets, (size_t)rankId); + // if (rankId == ccl_root) + // { + // itemOffsetsOnMaster = partialResult->get(training::init::offsets); + // } +} + +void initializeModel(size_t rankId, size_t partitionId, size_t nBlocks, size_t nUsers, + size_t nFactors) { + std::cout << "ALS (native): initializeModel " << std::endl; + + auto t1 = std::chrono::high_resolution_clock::now(); + + KeyValueDataCollectionPtr initStep1LocalResult = + initializeStep1Local(rankId, partitionId, nBlocks, nUsers, nFactors); + + /* MPI_Alltoallv to populate initStep2LocalInput */ + ByteBuffer nodeCPs[nBlocks]; + for (size_t i = 0; i < nBlocks; i++) { + serializeDAALObject((*initStep1LocalResult)[i].get(), nodeCPs[i]); + } + KeyValueDataCollectionPtr initStep2LocalInput(new KeyValueDataCollection()); + all2all(nodeCPs, nBlocks, initStep2LocalInput); + + initializeStep2Local(rankId, partitionId, initStep2LocalInput); + + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(t2 - t1).count(); + std::cout << "ALS (native): initializeModel took " << duration << " secs" << std::endl; +} + +training::DistributedPartialResultStep1Ptr computeStep1Local( + const training::DistributedPartialResultStep4Ptr& partialResultLocal, + size_t nFactors) { + /* Create algorithm objects to compute implicit ALS algorithm in the distributed + * processing mode on the local node using the default method */ + training::Distributed algorithm; + algorithm.parameter.nFactors = nFactors; + + /* Set input objects for the algorithm */ + algorithm.input.set(training::partialModel, + partialResultLocal->get(training::outputOfStep4ForStep1)); + + /* Compute partial estimates on local nodes */ + algorithm.compute(); + + /* Get the computed partial estimates */ + return algorithm.getPartialResult(); +} + +NumericTablePtr computeStep2Master( + const training::DistributedPartialResultStep1Ptr* step1LocalResultsOnMaster, + size_t nFactors, size_t nBlocks) { + /* Create algorithm objects to compute implicit ALS algorithm in the distributed + * processing mode on the master node using the default method */ + training::Distributed algorithm; + algorithm.parameter.nFactors = nFactors; + + /* Set input objects for the algorithm */ + for (size_t i = 0; i < nBlocks; i++) { + algorithm.input.add(training::inputOfStep2FromStep1, step1LocalResultsOnMaster[i]); + } + + /* Compute a partial estimate on the master node from the partial estimates on local + * nodes */ + algorithm.compute(); + + return algorithm.getPartialResult()->get(training::outputOfStep2ForStep4); +} + +KeyValueDataCollectionPtr computeStep3Local( + const NumericTablePtr& offset, + const training::DistributedPartialResultStep4Ptr& partialResultLocal, + const KeyValueDataCollectionPtr& step3LocalInput, size_t nFactors) { + training::Distributed algorithm; + algorithm.parameter.nFactors = nFactors; + + algorithm.input.set(training::partialModel, + partialResultLocal->get(training::outputOfStep4ForStep3)); + algorithm.input.set(training::inputOfStep3FromInit, step3LocalInput); + algorithm.input.set(training::offset, offset); + + algorithm.compute(); + + return algorithm.getPartialResult()->get(training::outputOfStep3ForStep4); +} + +training::DistributedPartialResultStep4Ptr computeStep4Local( + const CSRNumericTablePtr& dataTable, const NumericTablePtr& step2MasterResult, + const KeyValueDataCollectionPtr& step4LocalInput, size_t nFactors) { + training::Distributed algorithm; + algorithm.parameter.nFactors = nFactors; + + algorithm.input.set(training::partialModels, step4LocalInput); + algorithm.input.set(training::partialData, dataTable); + algorithm.input.set(training::inputOfStep4FromStep2, step2MasterResult); + + algorithm.compute(); + + return algorithm.getPartialResult(); +} + +void trainModel(size_t rankId, size_t partitionId, size_t nBlocks, size_t nFactors, + size_t maxIterations) { + std::cout << "ALS (native): trainModel" << std::endl; + + auto tStart = std::chrono::high_resolution_clock::now(); + + training::DistributedPartialResultStep1Ptr step1LocalResultsOnMaster[nBlocks]; + training::DistributedPartialResultStep1Ptr step1LocalResult; + NumericTablePtr step2MasterResult; + KeyValueDataCollectionPtr step3LocalResult; + KeyValueDataCollectionPtr step4LocalInput(new KeyValueDataCollection()); + + ByteBuffer nodeCPs[nBlocks]; + ByteBuffer nodeResults; + ByteBuffer crossProductBuf; + int crossProductLen; + + for (size_t iteration = 0; iteration < maxIterations; iteration++) { + auto t1 = std::chrono::high_resolution_clock::now(); + + // + // Update partial users factors + // + step1LocalResult = computeStep1Local(itemsPartialResultLocal, nFactors); + + serializeDAALObject(step1LocalResult.get(), nodeResults); + + /* Gathering step1LocalResult on the master */ + gather(rankId, nBlocks, nodeResults, step1LocalResultsOnMaster); + + if (rankId == ccl_root) { + step2MasterResult = + computeStep2Master(step1LocalResultsOnMaster, nFactors, nBlocks); + serializeDAALObject(step2MasterResult.get(), crossProductBuf); + crossProductLen = crossProductBuf.size(); + } + + ccl_request_t request; + + // MPI_Bcast(&crossProductLen, sizeof(int), MPI_CHAR, ccl_root, MPI_COMM_WORLD); + ccl_bcast(&crossProductLen, sizeof(int), ccl_dtype_char, ccl_root, NULL, NULL, NULL, + &request); + ccl_wait(request); + + if (rankId != ccl_root) { + crossProductBuf.resize(crossProductLen); + } + // MPI_Bcast(&crossProductBuf[0], crossProductLen, MPI_CHAR, ccl_root, + // MPI_COMM_WORLD); + ccl_bcast(&crossProductBuf[0], crossProductLen, ccl_dtype_char, ccl_root, NULL, NULL, + NULL, &request); + ccl_wait(request); + + step2MasterResult = + NumericTable::cast(deserializeDAALObject(&crossProductBuf[0], crossProductLen)); + + step3LocalResult = computeStep3Local(itemOffset, itemsPartialResultLocal, + itemStep3LocalInput, nFactors); + + /* MPI_Alltoallv to populate step4LocalInput */ + for (size_t i = 0; i < nBlocks; i++) { + serializeDAALObject((*step3LocalResult)[i].get(), nodeCPs[i]); + } + all2all(nodeCPs, nBlocks, step4LocalInput); + + usersPartialResultLocal = computeStep4Local(transposedDataTable, step2MasterResult, + step4LocalInput, nFactors); + + // + // Update partial items factors + // + step1LocalResult = computeStep1Local(usersPartialResultLocal, nFactors); + + serializeDAALObject(step1LocalResult.get(), nodeResults); + + /* Gathering step1LocalResult on the master */ + gather(rankId, nBlocks, nodeResults, step1LocalResultsOnMaster); + + if (rankId == ccl_root) { + step2MasterResult = + computeStep2Master(step1LocalResultsOnMaster, nFactors, nBlocks); + serializeDAALObject(step2MasterResult.get(), crossProductBuf); + crossProductLen = crossProductBuf.size(); + } + + // MPI_Bcast(&crossProductLen, sizeof(int), MPI_CHAR, ccl_root, MPI_COMM_WORLD); + ccl_bcast(&crossProductLen, sizeof(int), ccl_dtype_char, ccl_root, NULL, NULL, NULL, + &request); + ccl_wait(request); + + if (rankId != ccl_root) { + crossProductBuf.resize(crossProductLen); + } + + // MPI_Bcast(&crossProductBuf[0], crossProductLen, MPI_CHAR, ccl_root, + // MPI_COMM_WORLD); + ccl_bcast(&crossProductBuf[0], crossProductLen, ccl_dtype_char, ccl_root, NULL, NULL, + NULL, &request); + ccl_wait(request); + + step2MasterResult = + NumericTable::cast(deserializeDAALObject(&crossProductBuf[0], crossProductLen)); + + step3LocalResult = computeStep3Local(userOffset, usersPartialResultLocal, + userStep3LocalInput, nFactors); + + /* MPI_Alltoallv to populate step4LocalInput */ + for (size_t i = 0; i < nBlocks; i++) { + serializeDAALObject((*step3LocalResult)[i].get(), nodeCPs[i]); + } + all2all(nodeCPs, nBlocks, step4LocalInput); + + itemsPartialResultLocal = + computeStep4Local(dataTable, step2MasterResult, step4LocalInput, nFactors); + + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(t2 - t1).count(); + std::cout << "ALS (native): iteration " << iteration << " took " << duration + << " secs" << std::endl; + } + + auto tEnd = std::chrono::high_resolution_clock::now(); + auto durationTotal = + std::chrono::duration_cast(tEnd - tStart).count(); + std::cout << "ALS (native): trainModel took " << durationTotal << " secs" << std::endl; + + /*Gather all itemsPartialResultLocal to itemsPartialResultsMaster on the master and + * distributing the result over other ranks*/ + // serializeDAALObject(itemsPartialResultLocal.get(), nodeResults); + // gatherItems(nodeResults, nBlocks); + + // serializeDAALObject(usersPartialResultLocal.get(), nodeResults); + // gatherUsers(nodeResults, nBlocks); +} + +static size_t getOffsetFromOffsetTable(NumericTablePtr offsetTable) { + size_t ret; + BlockDescriptor block; + offsetTable->getBlockOfRows(0, 1, readOnly, block); + ret = (size_t)((block.getBlockPtr())[0]); + offsetTable->releaseBlockOfRows(block); + + 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( + JNIEnv* env, jobject obj, jobject dataBuffer, jint nTotalKeys, jint nBlocks, + jobject infoObj) { + // cout << "cShuffleData: rank " << rankId << endl; + cout << "RATING_SIZE: " << RATING_SIZE << endl; + + jbyte* ratingsBuf = (jbyte*)env->GetDirectBufferAddress(dataBuffer); + + jlong ratingsNum = env->GetDirectBufferCapacity(dataBuffer) / RATING_SIZE; + + std::vector ratingPartitions(nBlocks); + + for (int i = 0; i < ratingsNum; i++) { + Rating* rating = (Rating*)(ratingsBuf + RATING_SIZE * i); + int partition = getPartiton(rating->user, nTotalKeys, nBlocks); + ratingPartitions[partition].push_back(*rating); + } + + // for (int i = 0; i < nBlocks; i++) { + // cout << "Partition " << i << endl; + // for (auto r : ratingPartitions[i]) { + // cout << r.user << " " << r.item << " " << r.rating << endl; + // } + // } + + size_t newRatingsNum = 0; + size_t newCsrRowNum = 0; + Rating* ratings = shuffle_all2all(ratingPartitions, nBlocks, newRatingsNum, newCsrRowNum); + + // Get the class of the input object + jclass clazz = env->GetObjectClass(infoObj); + // Get Field references + jfieldID ratingsNumField = env->GetFieldID(clazz, "ratingsNum", "I"); + jfieldID csrRowNumField = env->GetFieldID(clazz, "csrRowNum", "I"); + + env->SetIntField(infoObj, ratingsNumField, newRatingsNum); + env->SetIntField(infoObj, csrRowNumField, newCsrRowNum); + + 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( + 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) { + + size_t rankId; + ccl_get_comm_rank(NULL, &rankId); + + dataTable = *((CSRNumericTablePtr*)numTableAddr); + // dataTable.reset(createFloatSparseTable("/home/xiaochang/github/oneDAL-upstream/samples/daal/cpp/mpi/data/distributed/implicit_als_csr_1.csv")); + + // printNumericTable(dataTable, "cDALImplictALS", 10); + cout << "ALS (native): Input info: " << endl; + cout << "- NumberOfRows: " << dataTable->getNumberOfRows() << endl; + cout << "- NumberOfColumns: " << dataTable->getNumberOfColumns() << endl; + cout << "- NumberOfRatings: " << dataTable->getDataSize() << endl; + cout << "- fullNUsers: " << nUsers << endl; + + // 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 threads used: " << nThreadsNew << endl; + + int nBlocks = executor_num; + initializeModel(rankId, partitionId, nBlocks, nUsers, nFactors); + trainModel(rankId, partitionId, executor_num, nFactors, maxIter); + + auto pUser = + usersPartialResultLocal->get(training::outputOfStep4ForStep1)->getFactors(); + // auto pUserIndices = + // usersPartialResultLocal->get(training::outputOfStep4ForStep1)->getIndices(); + auto pItem = + itemsPartialResultLocal->get(training::outputOfStep4ForStep1)->getFactors(); + // auto pItemIndices = + // itemsPartialResultsMaster[i]->get(training::outputOfStep4ForStep1)->getIndices(); + + std::cout << "\n=== Results for Rank " << rankId << "===\n" << std::endl; + // std::cout << "Partition ID: " << partitionId << std::endl; + printNumericTable(pUser, "User Factors (first 10 rows):", 10); + printNumericTable(pItem, "Item Factors (first 10 rows):", 10); + std::cout << "User Offset: " << getOffsetFromOffsetTable(userOffset) << std::endl; + std::cout << "Item Offset: " << getOffsetFromOffsetTable(itemOffset) << std::endl; + std::cout << std::endl; + + // printNumericTable(userOffset, "userOffset"); + // printNumericTable(itemOffset, "itemOffset"); + + // if (rankId == ccl_root) { + // for (int i = 0; i < nBlocks; i++) { + // printNumericTable(NumericTable::cast((*userOffsetsOnMaster)[i]), + // "userOffsetsOnMaster"); + // } + + // for (int i = 0; i < nBlocks; i++) { + // printNumericTable(NumericTable::cast((*itemOffsetsOnMaster)[i]), + // "itemOffsetsOnMaster"); + // } + // } + + // printf("native pUser %ld, pItem %ld", (jlong)&pUser, (jlong)&pItem); + + // Get the class of the input object + jclass clazz = env->GetObjectClass(resultObj); + + // Fill in rankId + jfieldID cRankIdField = env->GetFieldID(clazz, "rankId", "J"); + env->SetLongField(resultObj, cRankIdField, (jlong)rankId); + + // Fill in cUsersFactorsNumTab & cItemsFactorsNumTab + // Get Field references + jfieldID cUsersFactorsNumTabField = env->GetFieldID(clazz, "cUsersFactorsNumTab", "J"); + jfieldID cItemsFactorsNumTabField = env->GetFieldID(clazz, "cItemsFactorsNumTab", "J"); + // Set factors as result, should use heap memory + NumericTablePtr* retUser = new NumericTablePtr(pUser); + NumericTablePtr* retItem = new NumericTablePtr(pItem); + env->SetLongField(resultObj, cUsersFactorsNumTabField, (jlong)retUser); + env->SetLongField(resultObj, cItemsFactorsNumTabField, (jlong)retItem); + + // Fill in cUserOffset & cItemOffset + jfieldID cUserOffsetField = env->GetFieldID(clazz, "cUserOffset", "J"); + assert(cUserOffsetField != NULL); + env->SetLongField(resultObj, cUserOffsetField, + (jlong)getOffsetFromOffsetTable(userOffset)); + + jfieldID cItemOffsetField = env->GetFieldID(clazz, "cItemOffset", "J"); + assert(cItemOffsetField != NULL); + env->SetLongField(resultObj, cItemOffsetField, + (jlong)getOffsetFromOffsetTable(itemOffset)); + + return 0; +} diff --git a/mllib-dal/src/main/native/ALSShuffle.cpp b/mllib-dal/src/main/native/ALSShuffle.cpp new file mode 100644 index 000000000..ea83df3c2 --- /dev/null +++ b/mllib-dal/src/main/native/ALSShuffle.cpp @@ -0,0 +1,108 @@ +#include +#include +#include +#include +#include +#include + +#include "ALSShuffle.h" + +using namespace std; + +std::vector recvData; + +jlong getPartiton(jlong key, jlong totalKeys, long nBlocks) { + + jlong itemsInBlock = totalKeys / nBlocks; + + return min(key / itemsInBlock, nBlocks - 1); +} + +// Compares two Rating according to userId. +bool compareRatingByUser(Rating r1, Rating r2) +{ + if (r1.user < r2.user) + return true; + if (r1.user == r2.user && r1.item < r2.item) + return true; + return false; +} + +bool compareRatingUserEquality(Rating &r1, Rating &r2) { + return r1.user == r2.user; +} + +int distinct_count(std::vector &data) { + long curUser = -1; + long count = 0; + for (auto i : data) { + if (i.user > curUser) { + curUser = i.user; + count += 1; + } + } + return count; +} + +Rating * shuffle_all2all(std::vector &partitions, size_t nBlocks, size_t &newRatingsNum, size_t &newCsrRowNum) { + size_t sendBufSize = 0; + size_t recvBufSize = 0; + size_t perNodeSendLens[nBlocks]; + size_t perNodeRecvLens[nBlocks]; + + ByteBuffer sendData; + + size_t rankId; + ccl_get_comm_rank(NULL, &rankId); + + // Calculate send buffer size + for (size_t i = 0; i < nBlocks; i++) { + perNodeSendLens[i] = partitions[i].size() * RATING_SIZE; + // cout << "rank " << rankId << " Send partition " << i << " size " << perNodeSendLens[i] << endl; + sendBufSize += perNodeSendLens[i]; + } + cout << "sendData size " << sendBufSize << endl; + sendData.resize(sendBufSize); + + // Fill in send buffer + size_t offset = 0; + for (size_t i = 0; i < nBlocks; i++) + { + memcpy(sendData.data()+offset, partitions[i].data(), perNodeSendLens[i]); + offset += perNodeSendLens[i]; + } + + // Send lens first + ccl_request_t request; + ccl_alltoall(perNodeSendLens, perNodeRecvLens, sizeof(size_t), ccl_dtype_char, NULL, NULL, NULL, &request); + ccl_wait(request); + + // Calculate recv buffer size + for (size_t i = 0; i < nBlocks; i++) { + // cout << "rank " << rankId << " Recv partition " << i << " size " << perNodeRecvLens[i] << endl; + recvBufSize += perNodeRecvLens[i]; + } + + int ratingsNum = recvBufSize / RATING_SIZE; + recvData.resize(ratingsNum); + + // Send data + ccl_alltoallv(sendData.data(), perNodeSendLens, recvData.data(), perNodeRecvLens, ccl_dtype_char, NULL, NULL, NULL, &request); + ccl_wait(request); + + sort(recvData.begin(), recvData.end(), compareRatingByUser); + + // for (auto r : recvData) { + // cout << r.user << " " << r.item << " " << r.rating << endl; + // } + + newRatingsNum = recvData.size(); + // RatingPartition::iterator iter = std::unique(recvData.begin(), recvData.end(), compareRatingUserEquality); + // newCsrRowNum = std::distance(recvData.begin(), iter); + newCsrRowNum = distinct_count(recvData); + + cout << "newRatingsNum: " << newRatingsNum << " newCsrRowNum: " << newCsrRowNum << endl; + + return recvData.data(); +} + diff --git a/mllib-dal/src/main/native/ALSShuffle.h b/mllib-dal/src/main/native/ALSShuffle.h new file mode 100644 index 000000000..8e6917e4b --- /dev/null +++ b/mllib-dal/src/main/native/ALSShuffle.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +struct Rating { + jlong user; + jlong item; + jfloat rating; +} __attribute__((packed)); + +const int RATING_SIZE = sizeof(Rating); + +typedef std::vector ByteBuffer; +typedef std::vector RatingPartition; + +jlong getPartiton(jlong key, jlong totalKeys, long nBlocks); +Rating * shuffle_all2all(std::vector &partitions, size_t nBlocks, size_t &ratingsNum, size_t &csrRowNum); diff --git a/mllib-dal/src/main/native/Makefile b/mllib-dal/src/main/native/Makefile index bb071d6ec..ef2efcd94 100644 --- a/mllib-dal/src/main/native/Makefile +++ b/mllib-dal/src/main/native/Makefile @@ -38,10 +38,10 @@ LIBS := -L${CCL_ROOT}/lib -l:libccl.a \ # -L$(JAVA_HOME)/jre/lib/amd64 -ljsig CPP_SRCS += \ -./OneCCL.cpp ./OneDAL.cpp ./KMeansDALImpl.cpp ./PCADALImpl.cpp ./service.cpp ./error_handling.cpp +./OneCCL.cpp ./OneDAL.cpp ./KMeansDALImpl.cpp ./PCADALImpl.cpp ./ALSDALImpl.cpp ./ALSShuffle.cpp ./service.cpp ./error_handling.cpp OBJS += \ -./OneCCL.o ./OneDAL.o ./KMeansDALImpl.o ./PCADALImpl.o ./service.o ./error_handling.o +./OneCCL.o ./OneDAL.o ./KMeansDALImpl.o ./PCADALImpl.o ./ALSDALImpl.o ./ALSShuffle.o ./service.o ./error_handling.o # Output Binary OUTPUT = ../../../target/libMLlibDAL.so diff --git a/mllib-dal/src/main/native/OneCCL.cpp b/mllib-dal/src/main/native/OneCCL.cpp index b23be9737..21732b101 100644 --- a/mllib-dal/src/main/native/OneCCL.cpp +++ b/mllib-dal/src/main/native/OneCCL.cpp @@ -1,5 +1,6 @@ #include #include +#include #include "org_apache_spark_ml_util_OneCCL__.h" JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1init @@ -7,8 +8,14 @@ JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1init std::cout << "oneCCL (native): init" << std::endl; + auto t1 = std::chrono::high_resolution_clock::now(); + ccl_init(); + auto t2 = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast( t2 - t1 ).count(); + std::cout << "oneCCL (native): init took " << duration << " secs" << std::endl; + jclass cls = env->GetObjectClass(param); jfieldID fid_comm_size = env->GetFieldID(cls, "commSize", "J"); jfieldID fid_rank_id = env->GetFieldID(cls, "rankId", "J"); diff --git a/mllib-dal/src/main/native/OneDAL.cpp b/mllib-dal/src/main/native/OneDAL.cpp index 2210ffd1c..792225c3e 100644 --- a/mllib-dal/src/main/native/OneDAL.cpp +++ b/mllib-dal/src/main/native/OneDAL.cpp @@ -19,6 +19,8 @@ #include #include "org_apache_spark_ml_util_OneDAL__.h" +#include "service.h" + using namespace daal; using namespace daal::data_management; @@ -123,3 +125,51 @@ JNIEXPORT jboolean JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cCheckPlat // 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: cNewCSRNumericTable + * Signature: ([F[J[JJJ)J + */ +JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTable + (JNIEnv *env, jobject, jfloatArray data, jlongArray colIndices, jlongArray rowOffsets, jlong nFeatures, jlong nVectors) { + + long numData = env->GetArrayLength(data); + // long numColIndices = numData; + // long numRowOffsets = env->GetArrayLength(rowOffsets); + + size_t * resultRowOffsets = NULL; + size_t * resultColIndices = NULL; + float * resultData = NULL; + CSRNumericTable * numericTable = new CSRNumericTable(resultData, resultColIndices, resultRowOffsets, nFeatures, nVectors); + numericTable->allocateDataMemory(numData); + numericTable->getArrays(&resultData, &resultColIndices, &resultRowOffsets); + + size_t * pRowOffsets = (size_t *)env->GetLongArrayElements(rowOffsets, 0); + size_t * pColIndices = (size_t *)env->GetLongArrayElements(colIndices, 0); + float * pData = env->GetFloatArrayElements(data, 0); + + // std::memcpy(resultRowOffsets, pRowOffsets, numRowOffsets*sizeof(jlong)); + // std::memcpy(resultColIndices, pColIndices, numColIndices*sizeof(jlong)); + // std::memcpy(resultData, pData, numData*sizeof(float)); + + for (size_t i = 0; i < (size_t)numData; ++i) + { + resultData[i] = pData[i]; + resultColIndices[i] = pColIndices[i]; + } + for (size_t i = 0; i < (size_t)nVectors + 1; ++i) + { + resultRowOffsets[i] = pRowOffsets[i]; + } + + env->ReleaseLongArrayElements(rowOffsets, (jlong *)pRowOffsets, 0); + env->ReleaseLongArrayElements(colIndices, (jlong *)pColIndices, 0); + env->ReleaseFloatArrayElements(data, pData, 0); + + CSRNumericTablePtr *ret = new CSRNumericTablePtr(numericTable); + + //printNumericTable(*ret, "cNewCSRNumericTable", 10); + + return (jlong)ret; +} diff --git a/mllib-dal/src/main/native/build-jni.sh b/mllib-dal/src/main/native/build-jni.sh index bee614dcd..47a86b209 100755 --- a/mllib-dal/src/main/native/build-jni.sh +++ b/mllib-dal/src/main/native/build-jni.sh @@ -18,4 +18,6 @@ javah -d $WORK_DIR/javah -classpath "$WORK_DIR/../../../target/classes:$DAAL_JAR org.apache.spark.ml.util.OneCCL$ \ org.apache.spark.ml.util.OneDAL$ \ org.apache.spark.ml.clustering.KMeansDALImpl \ - org.apache.spark.ml.feature.PCADALImpl + org.apache.spark.ml.feature.PCADALImpl \ + org.apache.spark.ml.recommendation.ALSDALImpl + 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 new file mode 100644 index 000000000..73024d05b --- /dev/null +++ b/mllib-dal/src/main/native/javah/org_apache_spark_ml_recommendation_ALSDALImpl.h @@ -0,0 +1,29 @@ +/* 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_util_OneDAL__.h b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h index 5f67d9428..d88462d66 100644 --- a/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h +++ b/mllib-dal/src/main/native/javah/org_apache_spark_ml_util_OneDAL__.h @@ -47,6 +47,14 @@ JNIEXPORT void JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cFreeDataMemor JNIEXPORT jboolean JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cCheckPlatformCompatibility (JNIEnv *, jobject); +/* + * Class: org_apache_spark_ml_util_OneDAL__ + * Method: cNewCSRNumericTable + * Signature: ([F[J[JJJ)J + */ +JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_util_OneDAL_00024_cNewCSRNumericTable + (JNIEnv *, jobject, jfloatArray, jlongArray, jlongArray, jlong, jlong); + #ifdef __cplusplus } #endif diff --git a/mllib-dal/src/main/native/service.cpp b/mllib-dal/src/main/native/service.cpp index 9316b3b62..623767406 100644 --- a/mllib-dal/src/main/native/service.cpp +++ b/mllib-dal/src/main/native/service.cpp @@ -125,6 +125,10 @@ CSRNumericTable * createSparseTable(const std::string & datasetFileName) return numericTable; } +CSRNumericTable * createFloatSparseTable(const std::string & datasetFileName) { + return createSparseTable(datasetFileName); +} + void printAprioriItemsets(NumericTablePtr largeItemsetsTable, NumericTablePtr largeItemsetsSupportTable, size_t nItemsetToPrint = 20) { size_t largeItemsetCount = largeItemsetsSupportTable->getNumberOfRows(); diff --git a/mllib-dal/src/main/native/service.h b/mllib-dal/src/main/native/service.h index 37b702aea..b6a2cc5c5 100644 --- a/mllib-dal/src/main/native/service.h +++ b/mllib-dal/src/main/native/service.h @@ -43,5 +43,8 @@ typedef std::vector ByteBuffer; void printNumericTable(const NumericTablePtr & dataTable, const char * message = "", size_t nPrintedRows = 0, size_t nPrintedCols = 0, size_t interval = 10); +size_t serializeDAALObject(SerializationIface * pData, ByteBuffer & buffer); +SerializationIfacePtr deserializeDAALObject(daal::byte * buff, size_t length); +CSRNumericTable * createFloatSparseTable(const std::string & datasetFileName); #endif diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala new file mode 100644 index 000000000..9196873fb --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -0,0 +1,1885 @@ +/* + * 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 && isPlatformSupported) { + new ALSDALImpl(ratings, rank, maxIter, regParam, alpha, seed).run() + } 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/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala new file mode 100644 index 000000000..0e9dd4a35 --- /dev/null +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala @@ -0,0 +1,385 @@ +package org.apache.spark.ml.recommendation + +import com.intel.daal.data_management.data.CSRNumericTable.Indexing +import org.apache.spark.rdd.{ExecutorInProcessCoalescePartitioner, RDD} + +import scala.reflect.ClassTag +import com.intel.daal.data_management.data.{CSRNumericTable, HomogenNumericTable, RowMergedNumericTable, Matrix => DALMatrix} +import com.intel.daal.services.DaalContext +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._ + +import java.nio.{ByteBuffer, ByteOrder} +import scala.collection.mutable.ArrayBuffer +//import java.nio.DoubleBuffer +import java.nio.FloatBuffer + +class ALSDataPartitioner(blocks: Int, itemsInBlock: Long) + extends Partitioner { + def numPartitions: Int = blocks + def getPartition(key: Any): Int = { + val k = key.asInstanceOf[Long] + // itemsInBlock = numItems / partitions + // remaining records will belog to the last partition + // 21 => 5, 5, 5, 6 + // 46 => 11, 11, 11, 13 + math.min((k / itemsInBlock).toInt, blocks-1) + } +} + +class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( + data: RDD[Rating[ID]], + rank: Int, + maxIter: Int, + regParam: Double, + alpha: Double, + seed: Long, +) extends Serializable with Logging { + + // Rating struct size is size of Long+Long+Float + val RATING_SIZE = 8 + 8 + 4 + + // Return Map partitionId -> (ratingsNum, csrRowNum, rowOffset) + private def getRatingsPartitionInfo(data: RDD[Rating[ID]]): Map[Int, (Int, Int, Int)] = { + val collectd = data.mapPartitionsWithIndex { case (index: Int, it: Iterator[Rating[ID]]) => + var ratingsNum = 0 + var s = Set[ID]() + it.foreach { v => + s += v.user + ratingsNum += 1 + } + Iterator((index, (ratingsNum, s.count(_ => true)))) + }.collect + + var ret = Map[Int, (Int, Int, Int)]() + var rowOffset = 0 + collectd.foreach { v => + val partitionId = v._1 + val ratingsNum = v._2._1 + val csrRowNum = v._2._2 + ret += ( partitionId -> (ratingsNum, csrRowNum, rowOffset)) + rowOffset = rowOffset + csrRowNum + } + + ret + } + + private def ratingsToCSRNumericTables(ratings: RDD[Rating[ID]], + nVectors: Long, nFeatures: Long, nBlocks: Long): RDD[CSRNumericTable] = { + +// val rowSortedRatings = ratings.sortBy(_.user.toString.toLong) + +// val itemsInBlock = (nFeatures + nBlocks - 1) / nBlocks + val itemsInBlock = nFeatures / nBlocks +// val rowSortedGrouped = rowSortedRatings.groupBy(value => value.user.toString.toLong / itemsInBlock).flatMap(_._2) + val rowSortedGrouped = ratings + // Transpose the dataset + .map { p => + Rating(p.item, p.user, p.rating) + } + .groupBy(value => value.user.toString.toLong) + .partitionBy(new ALSDataPartitioner(nBlocks.toInt, itemsInBlock)) + .flatMap(_._2).mapPartitions { p => + p.toArray.sortBy(_.user.toString.toLong).toIterator + } + + println("rowSortedGrouped partition number: ", rowSortedGrouped.getNumPartitions) + + // rowSortedGrouped.mapPartitionsWithIndex { case (partitionId, partition) => +// println("partitionId", partitionId) +// partition.foreach { p => +// println(p.user, p.item, p.rating) } +// Iterator(partitionId) +// }.collect() + + val ratingsPartitionInfo = getRatingsPartitionInfo(rowSortedGrouped) + println("ratingsPartitionInfo:", ratingsPartitionInfo) + + rowSortedGrouped.mapPartitionsWithIndex { case (partitionId, partition) => + val ratingsNum = ratingsPartitionInfo(partitionId)._1 + val csrRowNum = ratingsPartitionInfo(partitionId)._2 + val values = Array.fill(ratingsNum) { 0.0f } + val columnIndices = Array.fill(ratingsNum) { 0L } + val rowOffsets = ArrayBuffer[Long](1L) + + + var index = 0 + var curRow = 0L + // Each partition converted to one CSRNumericTable + partition.foreach { p => + // Modify row index for each partition (start from 0) + val row = p.user.toString.toLong - ratingsPartitionInfo(partitionId)._3 + val column = p.item.toString.toLong + val rating = p.rating + + values(index) = rating + // one-based index + columnIndices(index) = column + 1 + + if (row > curRow) { + curRow = row + // one-based index + rowOffsets += index + 1 + } + + index = index + 1 + } + // one-based row index + rowOffsets += index+1 + + println("PartitionId:", partitionId) + println("csrRowNum", csrRowNum) +// println("rowOffsets", rowOffsets.mkString(",")) +// println("columnIndices", columnIndices.mkString(",")) +// println("values", values.mkString(",")) + + val contextLocal = new DaalContext() + + println("ALSDALImpl: Loading native libraries ..." ) + LibLoader.loadLibraries() + + val cTable = OneDAL.cNewCSRNumericTable(values, columnIndices, rowOffsets.toArray, nVectors, csrRowNum) + val table = new CSRNumericTable(contextLocal, cTable) +// table.pack() + + println("Input dimensions:", table.getNumberOfRows, table.getNumberOfColumns) + + // There is a bug https://github.com/oneapi-src/oneDAL/pull/1288, + // printNumericTable can't print correct result for CSRNumericTable, use C++ printNumericTable + // Service.printNumericTable("Input: ", table) + + Iterator(table) + }.cache() + } + +// def factorsToRDD(cUsersFactorsNumTab: Long, cItemsFactorsNumTab: Long) +// :(RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { +// val usersFactorsNumTab = OneDAL.makeNumericTable(cUsersFactorsNumTab) +// val itemsFactorsNumTab = OneDAL.makeNumericTable(cItemsFactorsNumTab) +// +// Service.printNumericTable("usersFactorsNumTab", usersFactorsNumTab) +// Service.printNumericTable("itemsFactorsNumTab", itemsFactorsNumTab) +// +// null +// } + + def ratingsToByteBuffer(ratings: Array[Rating[ID]]): ByteBuffer = { +// println("ratings len", ratings.length) + + val buffer= ByteBuffer.allocateDirect(ratings.length*(8+8+4)) + // Use little endian + buffer.order(ByteOrder.LITTLE_ENDIAN) + ratings.foreach { rating => + buffer.putLong(rating.user.toString.toLong) + buffer.putLong(rating.item.toString.toLong) + buffer.putFloat(rating.rating) + } + buffer + } + + def run(): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { + val executorNum = Utils.sparkExecutorNum(data.sparkContext) + val executorCores = Utils.sparkExecutorCores() + + val nFeatures = data.max()(new Ordering[Rating[ID]]() { + override def compare(x: Rating[ID], y: Rating[ID]): Int = + Ordering[Long].compare(x.item.toString.toLong, y.item.toString.toLong) + }).item.toString.toLong + 1 + + val nVectors = data.max()(new Ordering[Rating[ID]]() { + override def compare(x: Rating[ID], y: Rating[ID]): Int = + Ordering[Long].compare(x.user.toString.toLong, y.user.toString.toLong) + }).user.toString.toLong + 1 + +// val largestItems = data.sortBy(_.item.toString.toLong, ascending = false).take(1) +// val nFeatures = largestItems(0).item.toString.toLong + 1 + +// val largestUsers = data.sortBy(_.user.toString.toLong, ascending = false).take(1) +// val nVectors = largestUsers(0).user.toString.toLong + 1 + + val nBlocks = executorNum + +// val nRatings = data.count() + + logInfo(s"ALSDAL fit using $executorNum Executors for $nVectors vectors and $nFeatures features") + + val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) + + val numericTables = if (data.getNumPartitions < executorNum) { + data.repartition(executorNum).setName("Repartitioned for conversion").cache() + } else { + data.coalesce(executorNum).setName("Coalesced for conversion").cache() + } + + val results = numericTables + // Transpose the dataset + .map { p => + Rating(p.item, p.user, p.rating) } + .mapPartitions { iter => + val context = new DaalContext() + println("ALSDALImpl: Loading libMLlibDAL.so" ) + LibLoader.loadLibraries() + + OneCCL.init(executorNum, executorIPAddress, OneCCL.KVS_PORT) + val rankId = OneCCL.rankID() + + println("rankId", rankId, "nUsers", nVectors, "nItems", nFeatures) + + val buffer = ratingsToByteBuffer(iter.toArray) + val bufferInfo = new ALSPartitionInfo + val shuffledBuffer = cShuffleData(buffer, nFeatures.toInt, nBlocks, bufferInfo) + + val table = bufferToCSRNumericTable(shuffledBuffer, bufferInfo, nVectors.toInt, nFeatures.toInt, nBlocks, rankId) + + val result = new ALSResult() + cDALImplictALS( + table.getCNumericTable, nUsers = nVectors, + rank, maxIter, regParam, alpha, + executorNum, + executorCores, + rankId, + result + ) + Iterator(result) + }.cache() + +// results.foreach { p => +//// val usersFactorsNumTab = OneDAL.makeNumericTable(p.cUsersFactorsNumTab) +//// println("foreach", p.cUsersFactorsNumTab, p.cItemsFactorsNumTab) +// println("result", p.rankId, p.cUserOffset, p.cItemOffset); +// } + +// val usersFactorsRDD = results.mapPartitionsWithIndex { (index: Int, partiton: Iterator[ALSResult]) => +// partiton.foreach { p => +// val usersFactorsNumTab = OneDAL.makeNumericTable(p.cUsersFactorsNumTab) +// Service.printNumericTable("usersFactorsNumTab", usersFactorsNumTab) +// } +// Iterator() +// }.collect() + + val usersFactorsRDD = results.mapPartitionsWithIndex { (index: Int, partiton: Iterator[ALSResult]) => + val ret = partiton.flatMap { p => + val userOffset = p.cUserOffset.toInt + val usersFactorsNumTab = OneDAL.makeNumericTable(p.cUsersFactorsNumTab) + val nRows = usersFactorsNumTab.getNumberOfRows.toInt + val nCols = usersFactorsNumTab.getNumberOfColumns.toInt + var buffer = FloatBuffer.allocate(nCols * nRows) + // should use returned buffer + buffer = usersFactorsNumTab.getBlockOfRows(0, nRows, buffer) + (0 until nRows).map { index => + val array = Array.fill(nCols){0.0f} + buffer.get(array, 0, nCols) + ((index+userOffset).asInstanceOf[ID], array) + }.toIterator + } + ret + }.setName("userFactors").cache() + + val itemsFactorsRDD = results.mapPartitionsWithIndex { (index: Int, partiton: Iterator[ALSResult]) => + val ret = partiton.flatMap { p => + val itemOffset = p.cItemOffset.toInt + val itemsFactorsNumTab = OneDAL.makeNumericTable(p.cItemsFactorsNumTab) + val nRows = itemsFactorsNumTab.getNumberOfRows.toInt + val nCols = itemsFactorsNumTab.getNumberOfColumns.toInt + var buffer = FloatBuffer.allocate(nCols * nRows) + // should use returned buffer + buffer = itemsFactorsNumTab.getBlockOfRows(0, nRows, buffer) + (0 until nRows).map { index => + val array = Array.fill(nCols){0.0f} + buffer.get(array, 0, nCols) + ((index+itemOffset).asInstanceOf[ID], array) + }.toIterator + } + ret + }.setName("itemFactors").cache() + + usersFactorsRDD.count() + itemsFactorsRDD.count() + +// usersFactorsRDD.foreach { case (id, array) => +// println("usersFactorsRDD", id, array.mkString(", ")) +// } +// +// itemsFactorsRDD.foreach { case (id, array) => +// println("itemsFactorsRDD", id, array.mkString(", ")) +// } + + (usersFactorsRDD, itemsFactorsRDD) + } + + private def getPartitionOffset(partitionId: Int, nRatings: Int, nBlocks: Int): Int = { + require(partitionId >=0 && partitionId < nBlocks) + val itemsInBlock = nRatings / nBlocks + return partitionId * itemsInBlock + } + + private def bufferToCSRNumericTable(buffer: ByteBuffer, info: ALSPartitionInfo, + nVectors: Int, nFeatures: Int, nBlocks: Int, rankId: Int): CSRNumericTable = { + // Use little endian + buffer.order(ByteOrder.LITTLE_ENDIAN) + + val ratingsNum = info.ratingsNum + val csrRowNum = info.csrRowNum + val values = Array.fill(ratingsNum) { 0.0f } + val columnIndices = Array.fill(ratingsNum) { 0L } + val rowOffsets = ArrayBuffer[Long](1L) + + var index = 0 + var curRow = 0L + // Each partition converted to one CSRNumericTable + for (i <- 0 until ratingsNum) { + // Modify row index for each partition (start from 0) + val row = buffer.getLong(i*RATING_SIZE) - getPartitionOffset(rankId, nFeatures, nBlocks) + val column = buffer.getLong(i*RATING_SIZE+8) + val rating = buffer.getFloat(i*RATING_SIZE+16) + + values(index) = rating + // one-based index + columnIndices(index) = column + 1 + + if (row > curRow) { + curRow = row + // one-based index + rowOffsets += index + 1 + } + + index = index + 1 + } + // one-based row index + rowOffsets += index+1 + +// println("rankId:", rankId) +// println("csrRowNum", csrRowNum) + +// println(rowOffsets.mkString(" ")) +// println(columnIndices.mkString(" ")) +// println(values.mkString(" ")) + + val contextLocal = new DaalContext() + val cTable = OneDAL.cNewCSRNumericTable(values, columnIndices, rowOffsets.toArray, nVectors, csrRowNum) + val table = new CSRNumericTable(contextLocal, cTable) + + println("Input dimensions:", table.getNumberOfRows, table.getNumberOfColumns) +// Service.printNumericTable("Input NumericTable", table) + + table + } + + // Single entry to call Implict ALS DAL backend + @native private def cDALImplictALS(data: Long, + nUsers: Long, + rank: Int, + maxIter: Int, + regParam: Double, + alpha: Double, + executor_num: Int, + executor_cores: Int, + rankId: Int, + result: ALSResult): Long + @native private def cShuffleData(data: ByteBuffer, + nTotalKeys: Int, + nBlocks: Int, + info: ALSPartitionInfo): ByteBuffer +} \ No newline at end of file diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala index 2f6c83775..9b6c0f6c7 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneDAL.scala @@ -149,4 +149,7 @@ object OneDAL { @native def cFreeDataMemory(numTableAddr: Long) @native def cCheckPlatformCompatibility() : Boolean + + @native def cNewCSRNumericTable(data: Array[Float], colIndices: Array[Long], rowOffsets: Array[Long], nFeatures: Long, + nVectors: Long) : Long } From aaa8bc4676b87e2ca6fd5119f77bcc6ba24d7a72 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Tue, 2 Feb 2021 21:34:07 +0800 Subject: [PATCH 3/9] Add IntelALSSuite --- .../ml/recommendation/IntelALSSuite.scala | 1239 +++++++++++++++++ mllib-dal/test.sh | 3 +- 2 files changed, 1241 insertions(+), 1 deletion(-) create mode 100644 mllib-dal/src/test/scala/org/apache/spark/ml/recommendation/IntelALSSuite.scala diff --git a/mllib-dal/src/test/scala/org/apache/spark/ml/recommendation/IntelALSSuite.scala b/mllib-dal/src/test/scala/org/apache/spark/ml/recommendation/IntelALSSuite.scala new file mode 100644 index 000000000..339644888 --- /dev/null +++ b/mllib-dal/src/test/scala/org/apache/spark/ml/recommendation/IntelALSSuite.scala @@ -0,0 +1,1239 @@ +/* + * 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.io.File +import java.util.Random + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, WrappedArray} + +import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.recommendation.ALS._ +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} +import org.apache.spark.sql.{DataFrame, Encoder, Row, SparkSession} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.streaming.StreamingQueryException +import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + +class IntelALSSuite extends MLTest with DefaultReadWriteTest with Logging { + + override def beforeAll(): Unit = { + super.beforeAll() + sc.setCheckpointDir(tempDir.getAbsolutePath) + } + + override def afterAll(): Unit = { + super.afterAll() + } + + test("LocalIndexEncoder") { + val random = new Random + for (numBlocks <- Seq(1, 2, 5, 10, 20, 50, 100)) { + val encoder = new LocalIndexEncoder(numBlocks) + val maxLocalIndex = Int.MaxValue / numBlocks + val tests = Seq.fill(5)((random.nextInt(numBlocks), random.nextInt(maxLocalIndex))) ++ + Seq((0, 0), (numBlocks - 1, maxLocalIndex)) + tests.foreach { case (blockId, localIndex) => + val err = s"Failed with numBlocks=$numBlocks, blockId=$blockId, and localIndex=$localIndex." + val encoded = encoder.encode(blockId, localIndex) + assert(encoder.blockId(encoded) === blockId, err) + assert(encoder.localIndex(encoded) === localIndex, err) + } + } + } + + test("normal equation construction") { + val k = 2 + val ne0 = new NormalEquation(k) + .add(Array(1.0f, 2.0f), 3.0) + .add(Array(4.0f, 5.0f), 12.0, 2.0) // weighted + assert(ne0.k === k) + assert(ne0.triK === k * (k + 1) / 2) + // NumPy code that computes the expected values: + // A = np.matrix("1 2; 4 5") + // b = np.matrix("3; 6") + // C = np.matrix(np.diag([1, 2])) + // ata = A.transpose() * C * A + // atb = A.transpose() * C * b + assert(Vectors.dense(ne0.ata) ~== Vectors.dense(33.0, 42.0, 54.0) relTol 1e-8) + assert(Vectors.dense(ne0.atb) ~== Vectors.dense(51.0, 66.0) relTol 1e-8) + + val ne1 = new NormalEquation(2) + .add(Array(7.0f, 8.0f), 9.0) + ne0.merge(ne1) + // NumPy code that computes the expected values: + // A = np.matrix("1 2; 4 5; 7 8") + // b = np.matrix("3; 6; 9") + // C = np.matrix(np.diag([1, 2, 1])) + // ata = A.transpose() * C * A + // atb = A.transpose() * C * b + assert(Vectors.dense(ne0.ata) ~== Vectors.dense(82.0, 98.0, 118.0) relTol 1e-8) + assert(Vectors.dense(ne0.atb) ~== Vectors.dense(114.0, 138.0) relTol 1e-8) + + intercept[IllegalArgumentException] { + ne0.add(Array(1.0f), 2.0) + } + intercept[IllegalArgumentException] { + ne0.add(Array(1.0f, 2.0f, 3.0f), 4.0) + } + intercept[IllegalArgumentException] { + ne0.add(Array(1.0f, 2.0f), 0.0, -1.0) + } + intercept[IllegalArgumentException] { + val ne2 = new NormalEquation(3) + ne0.merge(ne2) + } + + ne0.reset() + assert(ne0.ata.forall(_ == 0.0)) + assert(ne0.atb.forall(_ == 0.0)) + } + + test("CholeskySolver") { + val k = 2 + val ne0 = new NormalEquation(k) + .add(Array(1.0f, 2.0f), 4.0) + .add(Array(1.0f, 3.0f), 9.0) + .add(Array(1.0f, 4.0f), 16.0) + val ne1 = new NormalEquation(k) + .merge(ne0) + + val chol = new CholeskySolver + val x0 = chol.solve(ne0, 0.0).map(_.toDouble) + // NumPy code that computes the expected solution: + // A = np.matrix("1 2; 1 3; 1 4") + // b = b = np.matrix("3; 6") + // x0 = np.linalg.lstsq(A, b)[0] + assert(Vectors.dense(x0) ~== Vectors.dense(-8.333333, 6.0) relTol 1e-6) + + assert(ne0.ata.forall(_ == 0.0)) + assert(ne0.atb.forall(_ == 0.0)) + + val x1 = chol.solve(ne1, 1.5).map(_.toDouble) + // NumPy code that computes the expected solution, where lambda is scaled by n: + // x0 = np.linalg.solve(A.transpose() * A + 1.5 * np.eye(2), A.transpose() * b) + assert(Vectors.dense(x1) ~== Vectors.dense(-0.1155556, 3.28) relTol 1e-6) + } + + test("RatingBlockBuilder") { + val emptyBuilder = new RatingBlockBuilder[Int]() + assert(emptyBuilder.size === 0) + val emptyBlock = emptyBuilder.build() + assert(emptyBlock.srcIds.isEmpty) + assert(emptyBlock.dstIds.isEmpty) + assert(emptyBlock.ratings.isEmpty) + + val builder0 = new RatingBlockBuilder() + .add(Rating(0, 1, 2.0f)) + .add(Rating(3, 4, 5.0f)) + assert(builder0.size === 2) + val builder1 = new RatingBlockBuilder() + .add(Rating(6, 7, 8.0f)) + .merge(builder0.build()) + assert(builder1.size === 3) + val block = builder1.build() + val ratings = Seq.tabulate(block.size) { i => + (block.srcIds(i), block.dstIds(i), block.ratings(i)) + }.toSet + assert(ratings === Set((0, 1, 2.0f), (3, 4, 5.0f), (6, 7, 8.0f))) + } + + test("UncompressedInBlock") { + val encoder = new LocalIndexEncoder(10) + val uncompressed = new UncompressedInBlockBuilder[Int](encoder) + .add(0, Array(1, 0, 2), Array(0, 1, 4), Array(1.0f, 2.0f, 3.0f)) + .add(1, Array(3, 0), Array(2, 5), Array(4.0f, 5.0f)) + .build() + assert(uncompressed.length === 5) + val records = Seq.tabulate(uncompressed.length) { i => + val dstEncodedIndex = uncompressed.dstEncodedIndices(i) + val dstBlockId = encoder.blockId(dstEncodedIndex) + val dstLocalIndex = encoder.localIndex(dstEncodedIndex) + (uncompressed.srcIds(i), dstBlockId, dstLocalIndex, uncompressed.ratings(i)) + }.toSet + val expected = + Set((1, 0, 0, 1.0f), (0, 0, 1, 2.0f), (2, 0, 4, 3.0f), (3, 1, 2, 4.0f), (0, 1, 5, 5.0f)) + assert(records === expected) + + val compressed = uncompressed.compress() + assert(compressed.size === 5) + assert(compressed.srcIds.toSeq === Seq(0, 1, 2, 3)) + assert(compressed.dstPtrs.toSeq === Seq(0, 2, 3, 4, 5)) + var decompressed = ArrayBuffer.empty[(Int, Int, Int, Float)] + var i = 0 + while (i < compressed.srcIds.length) { + var j = compressed.dstPtrs(i) + while (j < compressed.dstPtrs(i + 1)) { + val dstEncodedIndex = compressed.dstEncodedIndices(j) + val dstBlockId = encoder.blockId(dstEncodedIndex) + val dstLocalIndex = encoder.localIndex(dstEncodedIndex) + decompressed += ((compressed.srcIds(i), dstBlockId, dstLocalIndex, compressed.ratings(j))) + j += 1 + } + i += 1 + } + assert(decompressed.toSet === expected) + } + + test("CheckedCast") { + val checkedCast = new ALS().checkedCast + val df = spark.range(1) + + withClue("Valid Integer Ids") { + df.select(checkedCast(lit(123))).collect() + } + + withClue("Valid Long Ids") { + df.select(checkedCast(lit(1231L))).collect() + } + + withClue("Valid Decimal Ids") { + df.select(checkedCast(lit(123).cast(DecimalType(15, 2)))).collect() + } + + withClue("Valid Double Ids") { + df.select(checkedCast(lit(123.0))).collect() + } + + val msg = "either out of Integer range or contained a fractional part" + withClue("Invalid Long: out of range") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(1231000000000L))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Decimal: out of range") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(1231000000000.0).cast(DecimalType(15, 2)))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Decimal: fractional part") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(123.1).cast(DecimalType(15, 2)))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Double: out of range") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(1231000000000.0))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Double: fractional part") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(123.1))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Type") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit("123.1"))).collect() + } + assert(e.getMessage.contains("was not numeric")) + } + } + + /** + * Generates an explicit feedback dataset for testing ALS. + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genExplicitTestData( + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + val trainingFraction = 0.6 + val testFraction = 0.3 + val totalFraction = trainingFraction + testFraction + val random = new Random(seed) + val userFactors = genFactors(numUsers, rank, random) + val itemFactors = genFactors(numItems, rank, random) + val training = ArrayBuffer.empty[Rating[Int]] + val test = ArrayBuffer.empty[Rating[Int]] + for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { + val x = random.nextDouble() + if (x < totalFraction) { + val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) + if (x < trainingFraction) { + val noise = noiseStd * random.nextGaussian() + training += Rating(userId, itemId, rating + noise.toFloat) + } else { + test += Rating(userId, itemId, rating) + } + } + } + logInfo(s"Generated an explicit feedback dataset with ${training.size} ratings for training " + + s"and ${test.size} for test.") + (sc.parallelize(training, 2), sc.parallelize(test, 2)) + } + + /** + * Generates an implicit feedback dataset for testing ALS. + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genImplicitTestData( + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + ALSSuite.genImplicitTestData(sc, numUsers, numItems, rank, noiseStd, seed) + } + + /** + * Generates random user/item factors, with i.i.d. values drawn from U(a, b). + * @param size number of users/items + * @param rank number of features + * @param random random number generator + * @param a min value of the support (default: -1) + * @param b max value of the support (default: 1) + * @return a sequence of (ID, factors) pairs + */ + private def genFactors( + size: Int, + rank: Int, + random: Random, + a: Float = -1.0f, + b: Float = 1.0f): Seq[(Int, Array[Float])] = { + IntelALSSuite.genFactors(size, rank, random, a, b) + } + + /** + * Train ALS using the given training set and parameters + * @param training training dataset + * @param rank rank of the matrix factorization + * @param maxIter max number of iterations + * @param regParam regularization constant + * @param implicitPrefs whether to use implicit preference + * @param numUserBlocks number of user blocks + * @param numItemBlocks number of item blocks + * @return a trained ALSModel + */ + def trainALS( + training: RDD[Rating[Int]], + rank: Int, + maxIter: Int, + regParam: Double, + implicitPrefs: Boolean = false, + numUserBlocks: Int = 2, + numItemBlocks: Int = 3): ALSModel = { + val spark = this.spark + import spark.implicits._ + val als = new ALS() + .setRank(rank) + .setRegParam(regParam) + .setImplicitPrefs(implicitPrefs) + .setNumUserBlocks(numUserBlocks) + .setNumItemBlocks(numItemBlocks) + .setSeed(0) + als.fit(training.toDF()) + } + + /** + * Test ALS using the given training/test splits and parameters. + * @param training training dataset + * @param test test dataset + * @param rank rank of the matrix factorization + * @param maxIter max number of iterations + * @param regParam regularization constant + * @param implicitPrefs whether to use implicit preference + * @param numUserBlocks number of user blocks + * @param numItemBlocks number of item blocks + * @param targetRMSE target test RMSE + */ + def testALS( + training: RDD[Rating[Int]], + test: RDD[Rating[Int]], + rank: Int, + maxIter: Int, + regParam: Double, + implicitPrefs: Boolean = false, + numUserBlocks: Int = 2, + numItemBlocks: Int = 3, + targetRMSE: Double = 0.05): Unit = { + val spark = this.spark + import spark.implicits._ + val als = new ALS() + .setRank(rank) + .setRegParam(regParam) + .setImplicitPrefs(implicitPrefs) + .setNumUserBlocks(numUserBlocks) + .setNumItemBlocks(numItemBlocks) + .setSeed(0) + val alpha = als.getAlpha + val model = als.fit(training.toDF()) + testTransformerByGlobalCheckFunc[Rating[Int]](test.toDF(), model, "rating", "prediction") { + case rows: Seq[Row] => + val predictions = rows.map(row => (row.getFloat(0).toDouble, row.getFloat(1).toDouble)) + + val rmse = + if (implicitPrefs) { + // TODO: Use a better (rank-based?) evaluation metric for implicit feedback. + // We limit the ratings and the predictions to interval [0, 1] and compute the + // weighted RMSE with the confidence scores as weights. + val (totalWeight, weightedSumSq) = predictions.map { case (rating, prediction) => + val confidence = 1.0 + alpha * math.abs(rating) + val rating01 = math.max(math.min(rating, 1.0), 0.0) + val prediction01 = math.max(math.min(prediction, 1.0), 0.0) + val err = prediction01 - rating01 + (confidence, confidence * err * err) + }.reduce[(Double, Double)] { case ((c0, e0), (c1, e1)) => + (c0 + c1, e0 + e1) + } + math.sqrt(weightedSumSq / totalWeight) + } else { + val errorSquares = predictions.map { case (rating, prediction) => + val err = rating - prediction + err * err + } + val mse = errorSquares.sum / errorSquares.length + math.sqrt(mse) + } + logInfo(s"Test RMSE is $rmse.") + assert(rmse < targetRMSE) + } + + MLTestingUtils.checkCopyAndUids(als, model) + } + + test("exact rank-1 matrix") { + val (training, test) = genExplicitTestData(numUsers = 20, numItems = 40, rank = 1) + testALS(training, test, maxIter = 1, rank = 1, regParam = 1e-5, targetRMSE = 0.001) + testALS(training, test, maxIter = 1, rank = 2, regParam = 1e-5, targetRMSE = 0.001) + } + + test("approximate rank-1 matrix") { + val (training, test) = + genExplicitTestData(numUsers = 20, numItems = 40, rank = 1, noiseStd = 0.01) + testALS(training, test, maxIter = 2, rank = 1, regParam = 0.01, targetRMSE = 0.02) + testALS(training, test, maxIter = 2, rank = 2, regParam = 0.01, targetRMSE = 0.02) + } + + test("approximate rank-2 matrix") { + val (training, test) = + genExplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + testALS(training, test, maxIter = 4, rank = 2, regParam = 0.01, targetRMSE = 0.03) + testALS(training, test, maxIter = 4, rank = 3, regParam = 0.01, targetRMSE = 0.03) + } + + test("different block settings") { + val (training, test) = + genExplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + for ((numUserBlocks, numItemBlocks) <- Seq((1, 1), (1, 2), (2, 1), (2, 2))) { + testALS(training, test, maxIter = 4, rank = 3, regParam = 0.01, targetRMSE = 0.03, + numUserBlocks = numUserBlocks, numItemBlocks = numItemBlocks) + } + } + + test("more blocks than ratings") { + val (training, test) = + genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + testALS(training, test, maxIter = 2, rank = 1, regParam = 1e-4, targetRMSE = 0.002, + numItemBlocks = 5, numUserBlocks = 5) + } + + test("implicit feedback") { + val (training, test) = + genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + testALS(training, test, maxIter = 4, rank = 2, regParam = 0.01, implicitPrefs = true, + targetRMSE = 0.3) + } + + test("implicit feedback regression") { + val trainingWithNeg = sc.parallelize(Seq(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, -3))) + val trainingWithZero = sc.parallelize(Seq(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, 0))) + val modelWithNeg = + trainALS(trainingWithNeg, rank = 1, maxIter = 5, regParam = 0.01, implicitPrefs = true) + val modelWithZero = + trainALS(trainingWithZero, rank = 1, maxIter = 5, regParam = 0.01, implicitPrefs = true) + val userFactorsNeg = modelWithNeg.userFactors + val itemFactorsNeg = modelWithNeg.itemFactors + val userFactorsZero = modelWithZero.userFactors + val itemFactorsZero = modelWithZero.itemFactors + assert(userFactorsNeg.intersect(userFactorsZero).count() == 0) + assert(itemFactorsNeg.intersect(itemFactorsZero).count() == 0) + } + test("using generic ID types") { + val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + + val longRatings = ratings.map(r => Rating(r.user.toLong, r.item.toLong, r.rating)) + val (longUserFactors, _) = ALS.train(longRatings, rank = 2, maxIter = 4, seed = 0) + assert(longUserFactors.first()._1.getClass === classOf[Long]) + + val strRatings = ratings.map(r => Rating(r.user.toString, r.item.toString, r.rating)) + val (strUserFactors, _) = ALS.train(strRatings, rank = 2, maxIter = 4, seed = 0) + assert(strUserFactors.first()._1.getClass === classOf[String]) + } + + test("nonnegative constraint") { + val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + val (userFactors, itemFactors) = + ALS.train(ratings, rank = 2, maxIter = 4, nonnegative = true, seed = 0) + def isNonnegative(factors: RDD[(Int, Array[Float])]): Boolean = { + factors.values.map { _.forall(_ >= 0.0) }.reduce(_ && _) + } + assert(isNonnegative(userFactors)) + assert(isNonnegative(itemFactors)) + // TODO: Validate the solution. + } + + test("als partitioner is a projection") { + for (p <- Seq(1, 10, 100, 1000)) { + val part = new ALSPartitioner(p) + var k = 0 + while (k < p) { + assert(k === part.getPartition(k)) + assert(k === part.getPartition(k.toLong)) + k += 1 + } + } + } + + test("partitioner in returned factors") { + val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + val (userFactors, itemFactors) = ALS.train( + ratings, rank = 2, maxIter = 4, numUserBlocks = 3, numItemBlocks = 4, seed = 0) + for ((tpe, factors) <- Seq(("User", userFactors), ("Item", itemFactors))) { + assert(userFactors.partitioner.isDefined, s"$tpe factors should have partitioner.") + val part = userFactors.partitioner.get + userFactors.mapPartitionsWithIndex { (idx, items) => + items.foreach { case (id, _) => + if (part.getPartition(id) != idx) { + throw new SparkException(s"$tpe with ID $id should not be in partition $idx.") + } + } + Iterator.empty + }.count() + } + } + + test("als with large number of iterations") { + val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + ALS.train(ratings, rank = 1, maxIter = 50, numUserBlocks = 2, numItemBlocks = 2, seed = 0) + ALS.train(ratings, rank = 1, maxIter = 50, numUserBlocks = 2, numItemBlocks = 2, + implicitPrefs = true, seed = 0) + } + + test("read/write") { + val spark = this.spark + import ALSSuite._ + import spark.implicits._ + val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + + def getFactors(df: DataFrame): Set[(Int, Array[Float])] = { + df.select("id", "features").collect().map { case r => + (r.getInt(0), r.getAs[Array[Float]](1)) + }.toSet + } + + def checkModelData(model: ALSModel, model2: ALSModel): Unit = { + assert(model.rank === model2.rank) + assert(getFactors(model.userFactors) === getFactors(model2.userFactors)) + assert(getFactors(model.itemFactors) === getFactors(model2.itemFactors)) + } + + val als = new ALS() + testEstimatorAndModelReadWrite(als, ratings.toDF(), allEstimatorParamSettings, + allModelParamSettings, checkModelData) + } + + private def checkNumericTypesALS( + estimator: ALS, + spark: SparkSession, + column: String, + baseType: NumericType) + (check: (ALSModel, ALSModel) => Unit) + (check2: (ALSModel, ALSModel, DataFrame, Encoder[_]) => Unit): Unit = { + val dfs = genRatingsDFWithNumericCols(spark, column) + val maybeDf = dfs.find { case (numericTypeWithEncoder, _) => + numericTypeWithEncoder.numericType == baseType + } + assert(maybeDf.isDefined) + val df = maybeDf.get._2 + + val expected = estimator.fit(df) + val actuals = dfs.map(t => (t, estimator.fit(t._2))) + actuals.foreach { case (_, actual) => check(expected, actual) } + actuals.foreach { case (t, actual) => check2(expected, actual, t._2, t._1.encoder) } + + val baseDF = dfs.find(_._1.numericType == baseType).get._2 + val others = baseDF.columns.toSeq.diff(Seq(column)).map(col) + val cols = Seq(col(column).cast(StringType)) ++ others + val strDF = baseDF.select(cols: _*) + val thrown = intercept[IllegalArgumentException] { + estimator.fit(strDF) + } + assert(thrown.getMessage.contains( + s"$column must be of type numeric but was actually of type string")) + } + + private class NumericTypeWithEncoder[A](val numericType: NumericType) + (implicit val encoder: Encoder[(A, Int, Double)]) + + private def genRatingsDFWithNumericCols( + spark: SparkSession, + column: String) = { + + import testImplicits._ + + val df = spark.createDataFrame(Seq( + (0, 10, 1.0), + (1, 20, 2.0), + (2, 30, 3.0), + (3, 40, 4.0), + (4, 50, 5.0) + )).toDF("user", "item", "rating") + + val others = df.columns.toSeq.diff(Seq(column)).map(col) + val types = + Seq(new NumericTypeWithEncoder[Short](ShortType), + new NumericTypeWithEncoder[Long](LongType), + new NumericTypeWithEncoder[Int](IntegerType), + new NumericTypeWithEncoder[Float](FloatType), + new NumericTypeWithEncoder[Byte](ByteType), + new NumericTypeWithEncoder[Double](DoubleType), + new NumericTypeWithEncoder[Decimal](DecimalType(10, 0))(ExpressionEncoder()) + ) + types.map { t => + val cols = Seq(col(column).cast(t.numericType)) ++ others + t -> df.select(cols: _*) + } + } + + test("input type validation") { + val spark = this.spark + import spark.implicits._ + + // check that ALS can handle all numeric types for rating column + // and user/item columns (when the user/item ids are within Int range) + val als = new ALS().setMaxIter(1).setRank(1) + Seq(("user", IntegerType), ("item", IntegerType), ("rating", FloatType)).foreach { + case (colName, sqlType) => + checkNumericTypesALS(als, spark, colName, sqlType) { + (ex, act) => + ex.userFactors.first().getSeq[Float](1) === act.userFactors.first().getSeq[Float](1) + } { (ex, act, df, enc) => + // With AQE on/off, the order of result may be different. Here sortby the result. + val expected = ex.transform(df).selectExpr("prediction") + .sort("prediction").first().getFloat(0) + testTransformerByGlobalCheckFunc(df, act, "prediction") { + case rows: Seq[Row] => + expected ~== rows.sortBy(_.getFloat(0)).head.getFloat(0) absTol 1e-6 + }(enc) + } + } + // check user/item ids falling outside of Int range + val big = Int.MaxValue.toLong + 1 + val small = Int.MinValue.toDouble - 1 + val df = Seq( + (0, 0L, 0d, 1, 1L, 1d, 3.0), + (0, big, small, 0, big, small, 2.0), + (1, 1L, 1d, 0, 0L, 0d, 5.0) + ).toDF("user", "user_big", "user_small", "item", "item_big", "item_small", "rating") + val msg = "either out of Integer range or contained a fractional part" + withClue("fit should fail when ids exceed integer range. ") { + assert(intercept[SparkException] { + als.fit(df.select(df("user_big").as("user"), df("item"), df("rating"))) + }.getCause.getMessage.contains(msg)) + assert(intercept[SparkException] { + als.fit(df.select(df("user_small").as("user"), df("item"), df("rating"))) + }.getCause.getMessage.contains(msg)) + assert(intercept[SparkException] { + als.fit(df.select(df("item_big").as("item"), df("user"), df("rating"))) + }.getCause.getMessage.contains(msg)) + assert(intercept[SparkException] { + als.fit(df.select(df("item_small").as("item"), df("user"), df("rating"))) + }.getCause.getMessage.contains(msg)) + } + withClue("transform should fail when ids exceed integer range. ") { + val model = als.fit(df) + def testTransformIdExceedsIntRange[A : Encoder](dataFrame: DataFrame): Unit = { + val e1 = intercept[SparkException] { + model.transform(dataFrame).collect() + } + TestUtils.assertExceptionMsg(e1, msg) + val e2 = intercept[StreamingQueryException] { + testTransformer[A](dataFrame, model, "prediction") { _ => } + } + TestUtils.assertExceptionMsg(e2, msg) + } + testTransformIdExceedsIntRange[(Long, Int)](df.select(df("user_big").as("user"), + df("item"))) + testTransformIdExceedsIntRange[(Double, Int)](df.select(df("user_small").as("user"), + df("item"))) + testTransformIdExceedsIntRange[(Long, Int)](df.select(df("item_big").as("item"), + df("user"))) + testTransformIdExceedsIntRange[(Double, Int)](df.select(df("item_small").as("item"), + df("user"))) + } + } + + test("SPARK-18268: ALS with empty RDD should fail with better message") { + val ratings = sc.parallelize(Array.empty[Rating[Int]]) + intercept[IllegalArgumentException] { + ALS.train(ratings) + } + } + + test("ALS cold start user/item prediction strategy") { + val spark = this.spark + import org.apache.spark.sql.functions._ + import spark.implicits._ + + val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + val data = ratings.toDF + val knownUser = data.select(max("user")).as[Int].first() + val unknownUser = knownUser + 10 + val knownItem = data.select(max("item")).as[Int].first() + val unknownItem = knownItem + 20 + val test = Seq( + (unknownUser, unknownItem, true), + (knownUser, unknownItem, true), + (unknownUser, knownItem, true), + (knownUser, knownItem, false) + ).toDF("user", "item", "expectedIsNaN") + + val als = new ALS().setMaxIter(1).setRank(1) + // default is 'nan' + val defaultModel = als.fit(data) + testTransformer[(Int, Int, Boolean)](test, defaultModel, "expectedIsNaN", "prediction") { + case Row(expectedIsNaN: Boolean, prediction: Float) => + assert(prediction.isNaN === expectedIsNaN) + } + + // check 'drop' strategy should filter out rows with unknown users/items + val defaultPrediction = defaultModel.transform(test).select("prediction") + .as[Float].filter(!_.isNaN).first() + testTransformerByGlobalCheckFunc[(Int, Int, Boolean)](test, + defaultModel.setColdStartStrategy("drop"), "prediction") { + case rows: Seq[Row] => + val dropPredictions = rows.map(_.getFloat(0)) + assert(dropPredictions.length == 1) + assert(!dropPredictions.head.isNaN) + assert(dropPredictions.head ~== defaultPrediction relTol 1e-14) + } + } + + test("case insensitive cold start param value") { + val spark = this.spark + import spark.implicits._ + val (ratings, _) = genExplicitTestData(numUsers = 2, numItems = 2, rank = 1) + val data = ratings.toDF + val model = new ALS().fit(data) + Seq("nan", "NaN", "Nan", "drop", "DROP", "Drop").foreach { s => + testTransformer[Rating[Int]](data, model.setColdStartStrategy(s), "prediction") { _ => } + } + } + + private def getALSModel = { + val spark = this.spark + import spark.implicits._ + + val userFactors = Seq( + (0, Array(6.0f, 4.0f)), + (1, Array(3.0f, 4.0f)), + (2, Array(3.0f, 6.0f)) + ).toDF("id", "features") + val itemFactors = Seq( + (3, Array(5.0f, 6.0f)), + (4, Array(6.0f, 2.0f)), + (5, Array(3.0f, 6.0f)), + (6, Array(4.0f, 1.0f)) + ).toDF("id", "features") + val als = new ALS().setRank(2) + new ALSModel(als.uid, als.getRank, userFactors, itemFactors) + .setUserCol("user") + .setItemCol("item") + } + + test("recommendForAllUsers with k <, = and > num_items") { + val model = getALSModel + val numUsers = model.userFactors.count + val numItems = model.itemFactors.count + val expected = Map( + 0 -> Seq((3, 54f), (4, 44f), (5, 42f), (6, 28f)), + 1 -> Seq((3, 39f), (5, 33f), (4, 26f), (6, 16f)), + 2 -> Seq((3, 51f), (5, 45f), (4, 30f), (6, 18f)) + ) + + Seq(2, 4, 6).foreach { k => + val n = math.min(k, numItems).toInt + val expectedUpToN = expected.mapValues(_.slice(0, n)) + val topItems = model.recommendForAllUsers(k) + assert(topItems.count() == numUsers) + assert(topItems.columns.contains("user")) + checkRecommendations(topItems, expectedUpToN, "item") + } + } + + test("recommendForAllItems with k <, = and > num_users") { + val model = getALSModel + val numUsers = model.userFactors.count + val numItems = model.itemFactors.count + val expected = Map( + 3 -> Seq((0, 54f), (2, 51f), (1, 39f)), + 4 -> Seq((0, 44f), (2, 30f), (1, 26f)), + 5 -> Seq((2, 45f), (0, 42f), (1, 33f)), + 6 -> Seq((0, 28f), (2, 18f), (1, 16f)) + ) + + Seq(2, 3, 4).foreach { k => + val n = math.min(k, numUsers).toInt + val expectedUpToN = expected.mapValues(_.slice(0, n)) + val topUsers = getALSModel.recommendForAllItems(k) + assert(topUsers.count() == numItems) + assert(topUsers.columns.contains("item")) + checkRecommendations(topUsers, expectedUpToN, "user") + } + } + + test("recommendForUserSubset with k <, = and > num_items") { + val spark = this.spark + import spark.implicits._ + val model = getALSModel + val numItems = model.itemFactors.count + val expected = Map( + 0 -> Seq((3, 54f), (4, 44f), (5, 42f), (6, 28f)), + 2 -> Seq((3, 51f), (5, 45f), (4, 30f), (6, 18f)) + ) + val userSubset = expected.keys.toSeq.toDF("user") + val numUsersSubset = userSubset.count + + Seq(2, 4, 6).foreach { k => + val n = math.min(k, numItems).toInt + val expectedUpToN = expected.mapValues(_.slice(0, n)) + val topItems = model.recommendForUserSubset(userSubset, k) + assert(topItems.count() == numUsersSubset) + assert(topItems.columns.contains("user")) + checkRecommendations(topItems, expectedUpToN, "item") + } + } + + test("recommendForItemSubset with k <, = and > num_users") { + val spark = this.spark + import spark.implicits._ + val model = getALSModel + val numUsers = model.userFactors.count + val expected = Map( + 3 -> Seq((0, 54f), (2, 51f), (1, 39f)), + 6 -> Seq((0, 28f), (2, 18f), (1, 16f)) + ) + val itemSubset = expected.keys.toSeq.toDF("item") + val numItemsSubset = itemSubset.count + + Seq(2, 3, 4).foreach { k => + val n = math.min(k, numUsers).toInt + val expectedUpToN = expected.mapValues(_.slice(0, n)) + val topUsers = model.recommendForItemSubset(itemSubset, k) + assert(topUsers.count() == numItemsSubset) + assert(topUsers.columns.contains("item")) + checkRecommendations(topUsers, expectedUpToN, "user") + } + } + + test("subset recommendations eliminate duplicate ids, returns same results as unique ids") { + val spark = this.spark + import spark.implicits._ + val model = getALSModel + val k = 2 + + val users = Seq(0, 1).toDF("user") + val dupUsers = Seq(0, 1, 0, 1).toDF("user") + val singleUserRecs = model.recommendForUserSubset(users, k) + val dupUserRecs = model.recommendForUserSubset(dupUsers, k) + .as[(Int, Seq[(Int, Float)])].collect().toMap + assert(singleUserRecs.count == dupUserRecs.size) + checkRecommendations(singleUserRecs, dupUserRecs, "item") + + val items = Seq(3, 4, 5).toDF("item") + val dupItems = Seq(3, 4, 5, 4, 5).toDF("item") + val singleItemRecs = model.recommendForItemSubset(items, k) + val dupItemRecs = model.recommendForItemSubset(dupItems, k) + .as[(Int, Seq[(Int, Float)])].collect().toMap + assert(singleItemRecs.count == dupItemRecs.size) + checkRecommendations(singleItemRecs, dupItemRecs, "user") + } + + test("subset recommendations on full input dataset equivalent to recommendForAll") { + val spark = this.spark + import spark.implicits._ + val model = getALSModel + val k = 2 + + val userSubset = model.userFactors.withColumnRenamed("id", "user").drop("features") + val userSubsetRecs = model.recommendForUserSubset(userSubset, k) + val allUserRecs = model.recommendForAllUsers(k).as[(Int, Seq[(Int, Float)])].collect().toMap + checkRecommendations(userSubsetRecs, allUserRecs, "item") + + val itemSubset = model.itemFactors.withColumnRenamed("id", "item").drop("features") + val itemSubsetRecs = model.recommendForItemSubset(itemSubset, k) + val allItemRecs = model.recommendForAllItems(k).as[(Int, Seq[(Int, Float)])].collect().toMap + checkRecommendations(itemSubsetRecs, allItemRecs, "user") + } + + test("ALS should not introduce unnecessary shuffle") { + def getShuffledDependencies(rdd: RDD[_]): Seq[ShuffleDependency[_, _, _]] = { + rdd.dependencies.flatMap { + case s: ShuffleDependency[_, _, _] => + Seq(s) ++ getShuffledDependencies(s.rdd) + case o => + Seq.empty ++ getShuffledDependencies(o.rdd) + } + } + + val spark = this.spark + import spark.implicits._ + val (ratings, _) = genExplicitTestData(numUsers = 2, numItems = 2, rank = 1) + val data = ratings.toDF + val model = new ALS() + .setMaxIter(2) + .setImplicitPrefs(true) + .setCheckpointInterval(-1) + .fit(data) + + val userFactors = model.userFactors + val itemFactors = model.itemFactors + val shuffledUserFactors = getShuffledDependencies(userFactors.rdd).filter { dep => + dep.rdd.name != null && dep.rdd.name.contains("userFactors") + } + val shuffledItemFactors = getShuffledDependencies(itemFactors.rdd).filter { dep => + dep.rdd.name != null && dep.rdd.name.contains("itemFactors") + } + assert(shuffledUserFactors.size == 0) + assert(shuffledItemFactors.size == 0) + } + + private def checkRecommendations( + topK: DataFrame, + expected: Map[Int, Seq[(Int, Float)]], + dstColName: String): Unit = { + val spark = this.spark + import spark.implicits._ + + assert(topK.columns.contains("recommendations")) + topK.as[(Int, Seq[(Int, Float)])].collect().foreach { case (id: Int, recs: Seq[(Int, Float)]) => + assert(recs === expected(id)) + } + topK.collect().foreach { row => + val recs = row.getAs[WrappedArray[Row]]("recommendations") + assert(recs(0).fieldIndex(dstColName) == 0) + assert(recs(0).fieldIndex("rating") == 1) + } + } +} + +class ALSCleanerSuite extends SparkFunSuite with BeforeAndAfterEach { + override def beforeEach(): Unit = { + super.beforeEach() + // Once `Utils.getOrCreateLocalRootDirs` is called, it is cached in `Utils.localRootDirs`. + // Unless this is manually cleared before and after a test, it returns the same directory + // set before even if 'spark.local.dir' is configured afterwards. + Utils.clearLocalRootDirs() + } + + override def afterEach(): Unit = { + Utils.clearLocalRootDirs() + super.afterEach() + } + + test("ALS shuffle cleanup standalone") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Test checkpoint and clean parents + val input = sc.parallelize(1 to 1000) + val keyed = input.map(x => (x % 20, 1)) + val shuffled = keyed.reduceByKey(_ + _) + val keysOnly = shuffled.keys + val deps = keysOnly.dependencies + keysOnly.count() + ALS.cleanShuffleDependencies(sc, deps, true) + val resultingFiles = getAllFiles + assert(resultingFiles === Set()) + // Ensure running count again works fine even if we kill the shuffle files. + keysOnly.count() + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } + + test("ALS shuffle cleanup in algorithm") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "ALSCleanerSuite", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Generate test data + val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0) + // Implicitly test the cleaning of parents during ALS training + val spark = SparkSession.builder + .sparkContext(sc) + .getOrCreate() + import spark.implicits._ + val als = new ALS() + .setRank(1) + .setRegParam(1e-5) + .setSeed(0) + .setCheckpointInterval(1) + .setMaxIter(7) + val model = als.fit(training.toDF()) + val resultingFiles = getAllFiles + // We expect the last shuffles files, block ratings, user factors, and item factors to be + // around but no more. + val pattern = "shuffle_(\\d+)_.+\\.data".r + val rddIds = resultingFiles.flatMap { f => + pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } + assert(rddIds.size === 4) + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } +} + +class ALSStorageSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { + + test("invalid storage params") { + intercept[IllegalArgumentException] { + new ALS().setIntermediateStorageLevel("foo") + } + intercept[IllegalArgumentException] { + new ALS().setIntermediateStorageLevel("NONE") + } + intercept[IllegalArgumentException] { + new ALS().setFinalStorageLevel("foo") + } + } + + test("default and non-default storage params set correct RDD StorageLevels") { + val spark = this.spark + import spark.implicits._ + val data = Seq( + (0, 0, 1.0), + (0, 1, 2.0), + (1, 2, 3.0), + (1, 0, 2.0) + ).toDF("user", "item", "rating") + val als = new ALS().setMaxIter(1).setRank(1) + // add listener to check intermediate RDD default storage levels + val defaultListener = new IntermediateRDDStorageListener + sc.addSparkListener(defaultListener) + val model = als.fit(data) + // check final factor RDD default storage levels + val defaultFactorRDDs = sc.getPersistentRDDs.collect { + case (id, rdd) if rdd.name == "userFactors" || rdd.name == "itemFactors" => + rdd.name -> ((id, rdd.getStorageLevel)) + }.toMap + defaultFactorRDDs.foreach { case (_, (id, level)) => + assert(level == StorageLevel.MEMORY_AND_DISK) + } + defaultListener.storageLevels.foreach(level => assert(level == StorageLevel.MEMORY_AND_DISK)) + + // add listener to check intermediate RDD non-default storage levels + val nonDefaultListener = new IntermediateRDDStorageListener + sc.addSparkListener(nonDefaultListener) + val nonDefaultModel = als + .setFinalStorageLevel("MEMORY_ONLY") + .setIntermediateStorageLevel("DISK_ONLY") + .fit(data) + // check final factor RDD non-default storage levels + val levels = sc.getPersistentRDDs.collect { + case (id, rdd) if rdd.name == "userFactors" && rdd.id != defaultFactorRDDs("userFactors")._1 + || rdd.name == "itemFactors" && rdd.id != defaultFactorRDDs("itemFactors")._1 => + rdd.getStorageLevel + } + levels.foreach(level => assert(level == StorageLevel.MEMORY_ONLY)) + nonDefaultListener.storageLevels.foreach(level => assert(level == StorageLevel.DISK_ONLY)) + } +} + +private class IntermediateRDDStorageListener extends SparkListener { + + val storageLevels: mutable.ArrayBuffer[StorageLevel] = mutable.ArrayBuffer() + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + val stageLevels = stageCompleted.stageInfo.rddInfos.collect { + case info if info.name.contains("Blocks") || info.name.contains("Factors-") => + info.storageLevel + } + storageLevels ++= stageLevels + } + +} + +object IntelALSSuite extends Logging { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allModelParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPredictionCol" + ) + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allEstimatorParamSettings: Map[String, Any] = allModelParamSettings ++ Map( + "maxIter" -> 1, + "rank" -> 1, + "regParam" -> 0.01, + "numUserBlocks" -> 2, + "numItemBlocks" -> 2, + "implicitPrefs" -> true, + "alpha" -> 0.9, + "nonnegative" -> true, + "checkpointInterval" -> 20, + "intermediateStorageLevel" -> "MEMORY_ONLY", + "finalStorageLevel" -> "MEMORY_AND_DISK_SER" + ) + + // Helper functions to generate test data we share between ALS test suites + + /** + * Generates random user/item factors, with i.i.d. values drawn from U(a, b). + * @param size number of users/items + * @param rank number of features + * @param random random number generator + * @param a min value of the support (default: -1) + * @param b max value of the support (default: 1) + * @return a sequence of (ID, factors) pairs + */ + private def genFactors( + size: Int, + rank: Int, + random: Random, + a: Float = -1.0f, + b: Float = 1.0f): Seq[(Int, Array[Float])] = { + require(size > 0 && size < Int.MaxValue / 3) + require(b > a) + val ids = mutable.Set.empty[Int] + while (ids.size < size) { + ids += random.nextInt() + } + val width = b - a + ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + } + + /** + * Generates an implicit feedback dataset for testing ALS. + * + * @param sc SparkContext + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genImplicitTestData( + sc: SparkContext, + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + // The assumption of the implicit feedback model is that unobserved ratings are more likely to + // be negatives. + val positiveFraction = 0.8 + val negativeFraction = 1.0 - positiveFraction + val trainingFraction = 0.6 + val testFraction = 0.3 + val totalFraction = trainingFraction + testFraction + val random = new Random(seed) + val userFactors = genFactors(numUsers, rank, random) + val itemFactors = genFactors(numItems, rank, random) + val training = ArrayBuffer.empty[Rating[Int]] + val test = ArrayBuffer.empty[Rating[Int]] + for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { + val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) + val threshold = if (rating > 0) positiveFraction else negativeFraction + val observed = random.nextDouble() < threshold + if (observed) { + val x = random.nextDouble() + if (x < totalFraction) { + if (x < trainingFraction) { + val noise = noiseStd * random.nextGaussian() + training += Rating(userId, itemId, rating + noise.toFloat) + } else { + test += Rating(userId, itemId, rating) + } + } + } + } + logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + + s"and ${test.size} for test.") + (sc.parallelize(training, 2), sc.parallelize(test, 2)) + } +} diff --git a/mllib-dal/test.sh b/mllib-dal/test.sh index f7e73ca1f..839176fd3 100755 --- a/mllib-dal/test.sh +++ b/mllib-dal/test.sh @@ -35,8 +35,9 @@ export LD_PRELOAD=$JAVA_HOME/jre/lib/amd64/libjsig.so # -Dtest=none to turn off the Java tests # Test all -mvn -Dtest=none -Dmaven.test.skip=false test +#mvn -Dtest=none -Dmaven.test.skip=false test # Individual test # mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.clustering.IntelKMeansSuite test # mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.feature.IntelPCASuite test +mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.recommendation.IntelALSSuite test From 624a80ae14c57d0b2c2234a4f3461c62f382501e Mon Sep 17 00:00:00 2001 From: Xiaochang Wu Date: Wed, 3 Feb 2021 17:54:47 +0800 Subject: [PATCH 4/9] fix shuffle_all2all func declare --- mllib-dal/src/main/native/ALSDALImpl.cpp | 4 +++- mllib-dal/src/main/native/ALSShuffle.h | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib-dal/src/main/native/ALSDALImpl.cpp b/mllib-dal/src/main/native/ALSDALImpl.cpp index d1fb527af..4eb3ea973 100644 --- a/mllib-dal/src/main/native/ALSDALImpl.cpp +++ b/mllib-dal/src/main/native/ALSDALImpl.cpp @@ -531,6 +531,8 @@ JNIEXPORT jobject JNICALL Java_org_apache_spark_ml_recommendation_ALSDALImpl_cSh // cout << "cShuffleData: rank " << rankId << endl; cout << "RATING_SIZE: " << RATING_SIZE << endl; + ccl::communicator &comm = getComm(); + jbyte* ratingsBuf = (jbyte*)env->GetDirectBufferAddress(dataBuffer); jlong ratingsNum = env->GetDirectBufferCapacity(dataBuffer) / RATING_SIZE; @@ -552,7 +554,7 @@ JNIEXPORT jobject JNICALL Java_org_apache_spark_ml_recommendation_ALSDALImpl_cSh size_t newRatingsNum = 0; size_t newCsrRowNum = 0; - Rating* ratings = shuffle_all2all(ratingPartitions, nBlocks, newRatingsNum, newCsrRowNum); + Rating* ratings = shuffle_all2all(comm, ratingPartitions, nBlocks, newRatingsNum, newCsrRowNum); // Get the class of the input object jclass clazz = env->GetObjectClass(infoObj); diff --git a/mllib-dal/src/main/native/ALSShuffle.h b/mllib-dal/src/main/native/ALSShuffle.h index 8e6917e4b..dbe864978 100644 --- a/mllib-dal/src/main/native/ALSShuffle.h +++ b/mllib-dal/src/main/native/ALSShuffle.h @@ -14,4 +14,4 @@ typedef std::vector ByteBuffer; typedef std::vector RatingPartition; jlong getPartiton(jlong key, jlong totalKeys, long nBlocks); -Rating * shuffle_all2all(std::vector &partitions, size_t nBlocks, size_t &ratingsNum, size_t &csrRowNum); +Rating * shuffle_all2all(ccl::communicator &comm, std::vector &partitions, size_t nBlocks, size_t &ratingsNum, size_t &csrRowNum); From f8b82db09f0417aa27a7d3335052f04bbd8e78fd Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 3 Feb 2021 18:16:03 +0800 Subject: [PATCH 5/9] Rename ALS rank to nFactors and name conflict with oneCCL rank --- mllib-dal/src/main/native/ALSDALImpl.cpp | 1 + .../org/apache/spark/ml/recommendation/ALSDALImpl.scala | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/mllib-dal/src/main/native/ALSDALImpl.cpp b/mllib-dal/src/main/native/ALSDALImpl.cpp index 4eb3ea973..53212dc1d 100644 --- a/mllib-dal/src/main/native/ALSDALImpl.cpp +++ b/mllib-dal/src/main/native/ALSDALImpl.cpp @@ -591,6 +591,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_spark_ml_recommendation_ALSDALImpl_cDALI cout << "- NumberOfColumns: " << dataTable->getNumberOfColumns() << endl; cout << "- NumberOfRatings: " << dataTable->getDataSize() << endl; cout << "- fullNUsers: " << nUsers << endl; + cout << "- nFactors: " << nFactors << endl; // Set number of threads for oneDAL to use for each rank services::Environment::getInstance()->setNumberOfThreads(executor_cores); diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala index 1dd5276b6..7ba9e14e2 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala @@ -31,7 +31,7 @@ class ALSDataPartitioner(blocks: Int, itemsInBlock: Long) class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( data: RDD[Rating[ID]], - rank: Int, + nFactors: Int, maxIter: Int, regParam: Double, alpha: Double, @@ -237,7 +237,7 @@ class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( val result = new ALSResult() cDALImplictALS( table.getCNumericTable, nUsers = nVectors, - rank, maxIter, regParam, alpha, + nFactors, maxIter, regParam, alpha, executorNum, executorCores, rankId, @@ -371,7 +371,7 @@ class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( // Single entry to call Implict ALS DAL backend @native private def cDALImplictALS(data: Long, nUsers: Long, - rank: Int, + nFactors: Int, maxIter: Int, regParam: Double, alpha: Double, From 3624c07dd10bb8b63cf1e5d760090adcf0b34cb7 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Mon, 1 Mar 2021 16:08:58 +0800 Subject: [PATCH 6/9] Fix test.sh --- mllib-dal/test.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib-dal/test.sh b/mllib-dal/test.sh index 839176fd3..b4ae95035 100755 --- a/mllib-dal/test.sh +++ b/mllib-dal/test.sh @@ -35,9 +35,9 @@ export LD_PRELOAD=$JAVA_HOME/jre/lib/amd64/libjsig.so # -Dtest=none to turn off the Java tests # Test all -#mvn -Dtest=none -Dmaven.test.skip=false test +mvn -Dtest=none -Dmaven.test.skip=false test # Individual test # mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.clustering.IntelKMeansSuite test # mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.feature.IntelPCASuite test -mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.recommendation.IntelALSSuite test +# mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.recommendation.IntelALSSuite test From 5f88fb8de5cb38281127deb2b4b2d89590ae2de6 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Wed, 3 Mar 2021 16:24:45 +0800 Subject: [PATCH 7/9] use repartition to workaround partition uneven --- .../org/apache/spark/ml/recommendation/ALSDALImpl.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala index f84dc4beb..1e73b5308 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala @@ -213,12 +213,15 @@ class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( val kvsIPPort = kvsIP+"_"+kvsPort + val numericTables = data.repartition(executorNum).setName("Repartitioned for conversion").cache() + +/* val numericTables = if (data.getNumPartitions < executorNum) { data.repartition(executorNum).setName("Repartitioned for conversion").cache() } else { data.coalesce(executorNum).setName("Coalesced for conversion").cache() } - +*/ val results = numericTables // Transpose the dataset .map { p => @@ -388,4 +391,4 @@ class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( nTotalKeys: Int, nBlocks: Int, info: ALSPartitionInfo): ByteBuffer -} \ No newline at end of file +} From e4fcc269cc47312f1981422e573871afaa7de182 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Thu, 4 Mar 2021 13:52:13 +0800 Subject: [PATCH 8/9] Use getifaddr instead of hostname -I --- mllib-dal/src/main/native/OneCCL.cpp | 77 +++++++++++++++++++++++----- mllib-dal/test.sh | 6 +-- 2 files changed, 68 insertions(+), 15 deletions(-) diff --git a/mllib-dal/src/main/native/OneCCL.cpp b/mllib-dal/src/main/native/OneCCL.cpp index 3927968e6..0c8eb40c2 100644 --- a/mllib-dal/src/main/native/OneCCL.cpp +++ b/mllib-dal/src/main/native/OneCCL.cpp @@ -6,6 +6,10 @@ #include #include +#include +#include +#include + #include #include "org_apache_spark_ml_util_OneCCL__.h" @@ -112,22 +116,71 @@ JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_setEnv return err; } -#define GET_IP_CMD "hostname -I" -#define MAX_KVS_VAL_LENGTH 130 -#define READ_ONLY "r" +static const int CCL_IP_LEN = 128; +std::list local_host_ips; + +static int fill_local_host_ip() { + struct ifaddrs *ifaddr, *ifa; + int family = AF_UNSPEC; + char local_ip[CCL_IP_LEN]; + if (getifaddrs(&ifaddr) < 0) { + // LOG_ERROR("fill_local_host_ip: can not get host IP"); + return -1; + } + + const char iface_name[] = "lo"; + local_host_ips.clear(); + + for (ifa = ifaddr; ifa != NULL; ifa = ifa->ifa_next) { + if (ifa->ifa_addr == NULL) + continue; + if (strstr(ifa->ifa_name, iface_name) == NULL) { + family = ifa->ifa_addr->sa_family; + if (family == AF_INET) { + memset(local_ip, 0, CCL_IP_LEN); + int res = getnameinfo( + ifa->ifa_addr, + (family == AF_INET) ? sizeof(struct sockaddr_in) : sizeof(struct sockaddr_in6), + local_ip, + CCL_IP_LEN, + NULL, + 0, + NI_NUMERICHOST); + if (res != 0) { + std::string s("fill_local_host_ip: getnameinfo error > "); + s.append(gai_strerror(res)); + // LOG_ERROR(s.c_str()); + return -1; + } + local_host_ips.push_back(local_ip); + } + } + } + if (local_host_ips.empty()) { + // LOG_ERROR("fill_local_host_ip: can't find interface to get host IP"); + return -1; + } + // memset(local_host_ip, 0, CCL_IP_LEN); + // strncpy(local_host_ip, local_host_ips.front().c_str(), CCL_IP_LEN); + + // for (auto &ip : local_host_ips) + // cout << ip << endl; + + freeifaddrs(ifaddr); + return 0; +} static bool is_valid_ip(char ip[]) { - FILE *fp; - // TODO: use getifaddrs instead of popen - if ((fp = popen(GET_IP_CMD, READ_ONLY)) == NULL) { - printf("Can't get host IP\n"); - exit(1); + if (fill_local_host_ip() == -1) { + std::cerr << "fill_local_host_ip error" << std::endl; + }; + for (std::list::iterator it = local_host_ips.begin(); it != local_host_ips.end(); ++it) { + if (*it == ip) { + return true; } - char host_ips[MAX_KVS_VAL_LENGTH]; - fgets(host_ips, MAX_KVS_VAL_LENGTH, fp); - pclose(fp); + } - return strstr(host_ips, ip) ? true : false; + return false; } /* diff --git a/mllib-dal/test.sh b/mllib-dal/test.sh index b4ae95035..0157c22a4 100755 --- a/mllib-dal/test.sh +++ b/mllib-dal/test.sh @@ -35,9 +35,9 @@ export LD_PRELOAD=$JAVA_HOME/jre/lib/amd64/libjsig.so # -Dtest=none to turn off the Java tests # Test all -mvn -Dtest=none -Dmaven.test.skip=false test +# mvn -Dtest=none -Dmaven.test.skip=false test # Individual test -# mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.clustering.IntelKMeansSuite test -# mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.feature.IntelPCASuite test +mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.clustering.IntelKMeansSuite test +mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.feature.IntelPCASuite test # mvn -Dtest=none -DwildcardSuites=org.apache.spark.ml.recommendation.IntelALSSuite test From 98401ee4e078324fff637f8fbfde5bce6722286a Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Fri, 5 Mar 2021 14:18:49 +0800 Subject: [PATCH 9/9] add synchronized to getAvailPort and use dataForConversion --- mllib-dal/src/main/native/OneCCL.cpp | 2 +- .../javah/org_apache_spark_ml_util_OneCCL__.h | 4 ++-- .../spark/ml/clustering/KMeansDALImpl.scala | 15 +++++++-------- .../apache/spark/ml/feature/PCADALImpl.scala | 8 ++++---- .../spark/ml/recommendation/ALSDALImpl.scala | 19 ++++++------------- .../org/apache/spark/ml/util/OneCCL.scala | 6 +++++- 6 files changed, 25 insertions(+), 29 deletions(-) diff --git a/mllib-dal/src/main/native/OneCCL.cpp b/mllib-dal/src/main/native/OneCCL.cpp index 0c8eb40c2..c733c7b33 100644 --- a/mllib-dal/src/main/native/OneCCL.cpp +++ b/mllib-dal/src/main/native/OneCCL.cpp @@ -188,7 +188,7 @@ static bool is_valid_ip(char ip[]) { * Method: getAvailPort * Signature: (Ljava/lang/String;)I */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_getAvailPort +JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1getAvailPort (JNIEnv *env, jobject obj, jstring localIP) { // start from beginning of dynamic port 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 index 52e6691ee..580c34bf9 100644 --- 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 @@ -49,10 +49,10 @@ JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_setEnv /* * Class: org_apache_spark_ml_util_OneCCL__ - * Method: getAvailPort + * Method: c_getAvailPort * Signature: (Ljava/lang/String;)I */ -JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_getAvailPort +JNIEXPORT jint JNICALL Java_org_apache_spark_ml_util_OneCCL_00024_c_1getAvailPort (JNIEnv *, jobject, jstring); #ifdef __cplusplus diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala index f531b46a5..e9e7ec36d 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/clustering/KMeansDALImpl.scala @@ -40,14 +40,6 @@ class KMeansDALImpl ( instr.foreach(_.logInfo(s"Processing partitions with $executorNum executors")) - val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) - val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) - - val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) - val kvsPort = data.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) - - val kvsIPPort = kvsIP+"_"+kvsPort - // repartition to executorNum if not enough partitions val dataForConversion = if (data.getNumPartitions < executorNum) { data.repartition(executorNum).setName("Repartitioned for conversion").cache() @@ -55,6 +47,13 @@ class KMeansDALImpl ( data } + val executorIPAddress = Utils.sparkFirstExecutorIP(dataForConversion.sparkContext) + val kvsIP = dataForConversion.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) + val kvsPortDetected = Utils.checkExecutorAvailPort(dataForConversion, kvsIP) + val kvsPort = dataForConversion.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) + + val kvsIPPort = kvsIP+"_"+kvsPort + val partitionDims = Utils.getPartitionDims(dataForConversion) // filter the empty partitions diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala index 1b3f9ddf0..e1bba3d37 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/feature/PCADALImpl.scala @@ -47,11 +47,11 @@ class PCADALImpl ( val coalescedTables = OneDAL.rddVectorToNumericTables(normalizedData, executorNum) - val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) - val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) + val executorIPAddress = Utils.sparkFirstExecutorIP(coalescedTables.sparkContext) + val kvsIP = coalescedTables.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) - val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) - val kvsPort = data.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) + val kvsPortDetected = Utils.checkExecutorAvailPort(coalescedTables, kvsIP) + val kvsPort = coalescedTables.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) val kvsIPPort = kvsIP+"_"+kvsPort diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala index 1e73b5308..bcb95ca1f 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/recommendation/ALSDALImpl.scala @@ -205,23 +205,16 @@ class ALSDALImpl[@specialized(Int, Long) ID: ClassTag]( logInfo(s"ALSDAL fit using $executorNum Executors for $nVectors vectors and $nFeatures features") - val executorIPAddress = Utils.sparkFirstExecutorIP(data.sparkContext) - val kvsIP = data.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) + val numericTables = data.repartition(executorNum).setName("Repartitioned for conversion").cache() - val kvsPortDetected = Utils.checkExecutorAvailPort(data, kvsIP) - val kvsPort = data.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) + val executorIPAddress = Utils.sparkFirstExecutorIP(numericTables.sparkContext) + val kvsIP = numericTables.sparkContext.conf.get("spark.oap.mllib.oneccl.kvs.ip", executorIPAddress) - val kvsIPPort = kvsIP+"_"+kvsPort + val kvsPortDetected = Utils.checkExecutorAvailPort(numericTables, kvsIP) + val kvsPort = numericTables.sparkContext.conf.getInt("spark.oap.mllib.oneccl.kvs.port", kvsPortDetected) - val numericTables = data.repartition(executorNum).setName("Repartitioned for conversion").cache() + val kvsIPPort = kvsIP+"_"+kvsPort -/* - val numericTables = if (data.getNumPartitions < executorNum) { - data.repartition(executorNum).setName("Repartitioned for conversion").cache() - } else { - data.coalesce(executorNum).setName("Coalesced for conversion").cache() - } -*/ val results = numericTables // Transpose the dataset .map { p => diff --git a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala index 32b66a247..7581a1003 100644 --- a/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala +++ b/mllib-dal/src/main/scala/org/apache/spark/ml/util/OneCCL.scala @@ -50,6 +50,10 @@ object OneCCL extends Logging { c_cleanup() } + def getAvailPort(localIP: String): Int = synchronized { + c_getAvailPort(localIP) + } + @native private def c_init(size: Int, rank: Int, ip_port: String, param: CCLParam) : Int @native private def c_cleanup() : Unit @@ -57,5 +61,5 @@ object OneCCL extends Logging { @native def rankID() : Int @native def setEnv(key: String, value: String, overwrite: Boolean = true): Int - @native def getAvailPort(localIP: String): Int + @native def c_getAvailPort(localIP: String): Int } \ No newline at end of file