/
ALS.scala
1852 lines (1676 loc) · 69.6 KB
/
ALS.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.google.common.collect.{Ordering => GuavaOrdering}
import org.apache.hadoop.fs.Path
import org.json4s.DefaultFormats
import org.json4s.JsonDSL._
import org.apache.spark.{Partitioner, SparkException}
import org.apache.spark.annotation.Since
import org.apache.spark.internal.Logging
import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.linalg.BLAS
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared._
import org.apache.spark.ml.util._
import org.apache.spark.ml.util.DatasetUtils._
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._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
import org.apache.spark.storage.{StorageLevel, StorageLevelMapper}
import org.apache.spark.util.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] def checkIntegers(dataset: Dataset[_], colName: String): Column = {
dataset.schema(colName).dataType match {
case IntegerType =>
val column = dataset(colName)
when(column.isNull, raise_error(lit(s"$colName Ids MUST NOT be Null")))
.otherwise(column)
case _: NumericType =>
val column = dataset(colName)
val casted = column.cast(IntegerType)
// Checks if number within Int range and has no fractional part.
when(column.isNull || column =!= casted,
raise_error(concat(
lit(s"ALS only supports non-Null values in Integer range and " +
s"without fractional part for column $colName, but got "), column)))
.otherwise(casted)
case other => throw new IllegalArgumentException(s"ALS only supports values in " +
s"Integer range for column $colName, but got type $other.")
}
}
/**
* 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 -> StorageLevelMapper.MEMORY_AND_DISK.name(),
finalStorageLevel -> StorageLevelMapper.MEMORY_AND_DISK.name(), 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 validatedUsers = checkIntegers(dataset, $(userCol))
val validatedItems = checkIntegers(dataset, $(itemCol))
val validatedInputAlias = Identifiable.randomUID("__als_validated_input")
val itemFactorsAlias = Identifiable.randomUID("__als_item_factors")
val userFactorsAlias = Identifiable.randomUID("__als_user_factors")
val predictions = dataset
.withColumns(Seq($(userCol), $(itemCol)), Seq(validatedUsers, validatedItems))
.alias(validatedInputAlias)
.join(userFactors.alias(userFactorsAlias),
col(s"${validatedInputAlias}.${$(userCol)}") === col(s"${userFactorsAlias}.id"), "left")
.join(itemFactors.alias(itemFactorsAlias),
col(s"${validatedInputAlias}.${$(itemCol)}") === col(s"${itemFactorsAlias}.id"), "left")
.select(col(s"${validatedInputAlias}.*"),
predict(col(s"${userFactorsAlias}.features"), col(s"${itemFactorsAlias}.features"))
.alias($(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 GEMV (it use less memory compared with GEMM, and is much faster than DOT) and
* an efficient selection based on [[GuavaOrdering]] (instead of [[BoundedPriorityQueue]]).
* 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._
import scala.jdk.CollectionConverters._
val ratingColumn = "rating"
val recommendColumn = "recommendations"
val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize)
val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize)
val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked)
.as[(Array[Int], Array[Float], Array[Int], Array[Float])]
.mapPartitions { iter =>
var scores: Array[Float] = null
var idxOrd: GuavaOrdering[Int] = null
iter.flatMap { case (srcIds, srcMat, dstIds, dstMat) =>
require(srcMat.length == srcIds.length * rank)
require(dstMat.length == dstIds.length * rank)
val m = srcIds.length
val n = dstIds.length
if (scores == null || scores.length < n) {
scores = Array.ofDim[Float](n)
idxOrd = new GuavaOrdering[Int] {
override def compare(left: Int, right: Int): Int = {
Ordering[Float].compare(scores(left), scores(right))
}
}
}
Iterator.range(0, m).flatMap { i =>
// scores = i-th vec in srcMat * dstMat
BLAS.javaBLAS.sgemv("T", rank, n, 1.0F, dstMat, 0, rank,
srcMat, i * rank, 1, 0.0F, scores, 0, 1)
val srcId = srcIds(i)
idxOrd.greatestOf(Iterator.range(0, n).asJava, num).asScala
.iterator.map { j => (srcId, dstIds(j), scores(j)) }
}
}
}.toDF(srcOutputColumn, dstOutputColumn, ratingColumn)
val arrayType = ArrayType(
new StructType()
.add(dstOutputColumn, IntegerType)
.add(ratingColumn, FloatType)
)
ratings.groupBy(srcOutputColumn)
.agg(collect_top_k(struct(ratingColumn, dstOutputColumn), num, false))
.as[(Int, Seq[(Float, Int)])]
.map(t => (t._1, t._2.map(p => (p._2, p._1))))
.toDF(srcOutputColumn, recommendColumn)
.withColumn(recommendColumn, col(recommendColumn).cast(arrayType))
}
/**
* Blockifies factors to improve the efficiency of cross join
*/
private def blockify(
factors: Dataset[(Int, Array[Float])],
blockSize: Int): Dataset[(Array[Int], Array[Float])] = {
import factors.sparkSession.implicits._
factors.mapPartitions { iter =>
iter.grouped(blockSize)
.map(block => (block.map(_._1).toArray, block.flatMap(_._2).toArray))
}
}
}
@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 validatedUsers = checkIntegers(dataset, $(userCol))
val validatedItems = checkIntegers(dataset, $(itemCol))
val validatedRatings = if ($(ratingCol).nonEmpty) {
checkNonNanValues($(ratingCol), "Ratings").cast(FloatType)
} else {
lit(1.0f)
}
val ratings = dataset
.select(validatedUsers, validatedItems, validatedRatings)
.rdd
.map { case Row(u: Int, i: Int, r: Float) => Rating(u, i, r) }
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.nativeBLAS.dspr(upper, k, c, da, 1, ata)
if (b != 0.0) {
BLAS.nativeBLAS.daxpy(k, b, da, 1, atb, 1)
}
this
}
/** Merges another normal equation object. */
def merge(other: NormalEquation): NormalEquation = {
require(other.k == k)
BLAS.nativeBLAS.daxpy(ata.length, 1.0, other.ata, 1, ata, 1)
BLAS.nativeBLAS.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)
}
}
/**
* 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.
*/
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])]) = {
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))