-
Notifications
You must be signed in to change notification settings - Fork 28k
/
recommendation.py
748 lines (656 loc) · 24.2 KB
/
recommendation.py
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
#
# 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.
#
import sys
from typing import Any, Dict, Optional, TYPE_CHECKING
from pyspark import since, keyword_only
from pyspark.ml.param.shared import (
HasPredictionCol,
HasBlockSize,
HasMaxIter,
HasRegParam,
HasCheckpointInterval,
HasSeed,
)
from pyspark.ml.wrapper import JavaEstimator, JavaModel
from pyspark.ml.common import inherit_doc
from pyspark.ml.param import Params, TypeConverters, Param
from pyspark.ml.util import JavaMLWritable, JavaMLReadable
from pyspark.sql import DataFrame
if TYPE_CHECKING:
from py4j.java_gateway import JavaObject
__all__ = ["ALS", "ALSModel"]
@inherit_doc
class _ALSModelParams(HasPredictionCol, HasBlockSize):
"""
Params for :py:class:`ALS` and :py:class:`ALSModel`.
.. versionadded:: 3.0.0
"""
userCol: Param[str] = Param(
Params._dummy(),
"userCol",
"column name for user ids. Ids must be within " + "the integer value range.",
typeConverter=TypeConverters.toString,
)
itemCol: Param[str] = Param(
Params._dummy(),
"itemCol",
"column name for item ids. Ids must be within " + "the integer value range.",
typeConverter=TypeConverters.toString,
)
coldStartStrategy: Param[str] = Param(
Params._dummy(),
"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: 'nan', 'drop'.",
typeConverter=TypeConverters.toString,
)
def __init__(self, *args: Any):
super(_ALSModelParams, self).__init__(*args)
self._setDefault(blockSize=4096)
@since("1.4.0")
def getUserCol(self) -> str:
"""
Gets the value of userCol or its default value.
"""
return self.getOrDefault(self.userCol)
@since("1.4.0")
def getItemCol(self) -> str:
"""
Gets the value of itemCol or its default value.
"""
return self.getOrDefault(self.itemCol)
@since("2.2.0")
def getColdStartStrategy(self) -> str:
"""
Gets the value of coldStartStrategy or its default value.
"""
return self.getOrDefault(self.coldStartStrategy)
@inherit_doc
class _ALSParams(_ALSModelParams, HasMaxIter, HasRegParam, HasCheckpointInterval, HasSeed):
"""
Params for :py:class:`ALS`.
.. versionadded:: 3.0.0
"""
rank: Param[int] = Param(
Params._dummy(), "rank", "rank of the factorization", typeConverter=TypeConverters.toInt
)
numUserBlocks: Param[int] = Param(
Params._dummy(),
"numUserBlocks",
"number of user blocks",
typeConverter=TypeConverters.toInt,
)
numItemBlocks: Param[int] = Param(
Params._dummy(),
"numItemBlocks",
"number of item blocks",
typeConverter=TypeConverters.toInt,
)
implicitPrefs: Param[bool] = Param(
Params._dummy(),
"implicitPrefs",
"whether to use implicit preference",
typeConverter=TypeConverters.toBoolean,
)
alpha: Param[float] = Param(
Params._dummy(),
"alpha",
"alpha for implicit preference",
typeConverter=TypeConverters.toFloat,
)
ratingCol: Param[str] = Param(
Params._dummy(),
"ratingCol",
"column name for ratings",
typeConverter=TypeConverters.toString,
)
nonnegative: Param[bool] = Param(
Params._dummy(),
"nonnegative",
"whether to use nonnegative constraint for least squares",
typeConverter=TypeConverters.toBoolean,
)
intermediateStorageLevel: Param[str] = Param(
Params._dummy(),
"intermediateStorageLevel",
"StorageLevel for intermediate datasets. Cannot be 'NONE'.",
typeConverter=TypeConverters.toString,
)
finalStorageLevel: Param[str] = Param(
Params._dummy(),
"finalStorageLevel",
"StorageLevel for ALS model factors.",
typeConverter=TypeConverters.toString,
)
def __init__(self, *args: Any):
super(_ALSParams, self).__init__(*args)
self._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",
)
@since("1.4.0")
def getRank(self) -> int:
"""
Gets the value of rank or its default value.
"""
return self.getOrDefault(self.rank)
@since("1.4.0")
def getNumUserBlocks(self) -> int:
"""
Gets the value of numUserBlocks or its default value.
"""
return self.getOrDefault(self.numUserBlocks)
@since("1.4.0")
def getNumItemBlocks(self) -> int:
"""
Gets the value of numItemBlocks or its default value.
"""
return self.getOrDefault(self.numItemBlocks)
@since("1.4.0")
def getImplicitPrefs(self) -> bool:
"""
Gets the value of implicitPrefs or its default value.
"""
return self.getOrDefault(self.implicitPrefs)
@since("1.4.0")
def getAlpha(self) -> float:
"""
Gets the value of alpha or its default value.
"""
return self.getOrDefault(self.alpha)
@since("1.4.0")
def getRatingCol(self) -> str:
"""
Gets the value of ratingCol or its default value.
"""
return self.getOrDefault(self.ratingCol)
@since("1.4.0")
def getNonnegative(self) -> bool:
"""
Gets the value of nonnegative or its default value.
"""
return self.getOrDefault(self.nonnegative)
@since("2.0.0")
def getIntermediateStorageLevel(self) -> str:
"""
Gets the value of intermediateStorageLevel or its default value.
"""
return self.getOrDefault(self.intermediateStorageLevel)
@since("2.0.0")
def getFinalStorageLevel(self) -> str:
"""
Gets the value of finalStorageLevel or its default value.
"""
return self.getOrDefault(self.finalStorageLevel)
@inherit_doc
class ALS(JavaEstimator["ALSModel"], _ALSParams, JavaMLWritable, JavaMLReadable["ALS"]):
"""
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",
<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 > 0 and 0 if r <= 0.
The ratings then act as 'confidence' values related to strength of
indicated user preferences rather than explicit ratings given to
items.
.. versionadded:: 1.4.0
Notes
-----
The input rating dataframe 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
dataframe output nondeterministic, like `df.repartition(2).sample(False, 0.5, 1618)`.
Checkpointing sampled dataframe or adding a sort before sampling can help make the
dataframe deterministic.
Examples
--------
>>> df = spark.createDataFrame(
... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)],
... ["user", "item", "rating"])
>>> als = ALS(rank=10, seed=0)
>>> als.setMaxIter(5)
ALS...
>>> als.getMaxIter()
5
>>> als.setRegParam(0.1)
ALS...
>>> als.getRegParam()
0.1
>>> als.clear(als.regParam)
>>> model = als.fit(df)
>>> model.getBlockSize()
4096
>>> model.getUserCol()
'user'
>>> model.setUserCol("user")
ALSModel...
>>> model.getItemCol()
'item'
>>> model.setPredictionCol("newPrediction")
ALS...
>>> model.rank
10
>>> model.userFactors.orderBy("id").collect()
[Row(id=0, features=[...]), Row(id=1, ...), Row(id=2, ...)]
>>> test = spark.createDataFrame([(0, 2), (1, 0), (2, 0)], ["user", "item"])
>>> predictions = sorted(model.transform(test).collect(), key=lambda r: r[0])
>>> predictions[0]
Row(user=0, item=2, newPrediction=0.6929...)
>>> predictions[1]
Row(user=1, item=0, newPrediction=3.47356...)
>>> predictions[2]
Row(user=2, item=0, newPrediction=-0.899198...)
>>> user_recs = model.recommendForAllUsers(3)
>>> user_recs.where(user_recs.user == 0)\
.select("recommendations.item", "recommendations.rating").collect()
[Row(item=[0, 1, 2], rating=[3.910..., 1.997..., 0.692...])]
>>> item_recs = model.recommendForAllItems(3)
>>> item_recs.where(item_recs.item == 2)\
.select("recommendations.user", "recommendations.rating").collect()
[Row(user=[2, 1, 0], rating=[4.892..., 3.991..., 0.692...])]
>>> user_subset = df.where(df.user == 2)
>>> user_subset_recs = model.recommendForUserSubset(user_subset, 3)
>>> user_subset_recs.select("recommendations.item", "recommendations.rating").first()
Row(item=[2, 1, 0], rating=[4.892..., 1.076..., -0.899...])
>>> item_subset = df.where(df.item == 0)
>>> item_subset_recs = model.recommendForItemSubset(item_subset, 3)
>>> item_subset_recs.select("recommendations.user", "recommendations.rating").first()
Row(user=[0, 1, 2], rating=[3.910..., 3.473..., -0.899...])
>>> als_path = temp_path + "/als"
>>> als.save(als_path)
>>> als2 = ALS.load(als_path)
>>> als.getMaxIter()
5
>>> model_path = temp_path + "/als_model"
>>> model.save(model_path)
>>> model2 = ALSModel.load(model_path)
>>> model.rank == model2.rank
True
>>> sorted(model.userFactors.collect()) == sorted(model2.userFactors.collect())
True
>>> sorted(model.itemFactors.collect()) == sorted(model2.itemFactors.collect())
True
>>> model.transform(test).take(1) == model2.transform(test).take(1)
True
"""
_input_kwargs: Dict[str, Any]
@keyword_only
def __init__(
self,
*,
rank: int = 10,
maxIter: int = 10,
regParam: float = 0.1,
numUserBlocks: int = 10,
numItemBlocks: int = 10,
implicitPrefs: bool = False,
alpha: float = 1.0,
userCol: str = "user",
itemCol: str = "item",
seed: Optional[int] = None,
ratingCol: str = "rating",
nonnegative: bool = False,
checkpointInterval: int = 10,
intermediateStorageLevel: str = "MEMORY_AND_DISK",
finalStorageLevel: str = "MEMORY_AND_DISK",
coldStartStrategy: str = "nan",
blockSize: int = 4096,
):
"""
__init__(self, \\*, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10,
numItemBlocks=10, implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", \
seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, \
intermediateStorageLevel="MEMORY_AND_DISK", \
finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096)
"""
super(ALS, self).__init__()
self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid)
kwargs = self._input_kwargs
self.setParams(**kwargs)
@keyword_only
@since("1.4.0")
def setParams(
self,
*,
rank: int = 10,
maxIter: int = 10,
regParam: float = 0.1,
numUserBlocks: int = 10,
numItemBlocks: int = 10,
implicitPrefs: bool = False,
alpha: float = 1.0,
userCol: str = "user",
itemCol: str = "item",
seed: Optional[int] = None,
ratingCol: str = "rating",
nonnegative: bool = False,
checkpointInterval: int = 10,
intermediateStorageLevel: str = "MEMORY_AND_DISK",
finalStorageLevel: str = "MEMORY_AND_DISK",
coldStartStrategy: str = "nan",
blockSize: int = 4096,
) -> "ALS":
"""
setParams(self, \\*, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, \
numItemBlocks=10, implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", \
seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, \
intermediateStorageLevel="MEMORY_AND_DISK", \
finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096)
Sets params for ALS.
"""
kwargs = self._input_kwargs
return self._set(**kwargs)
def _create_model(self, java_model: "JavaObject") -> "ALSModel":
return ALSModel(java_model)
@since("1.4.0")
def setRank(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`rank`.
"""
return self._set(rank=value)
@since("1.4.0")
def setNumUserBlocks(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`numUserBlocks`.
"""
return self._set(numUserBlocks=value)
@since("1.4.0")
def setNumItemBlocks(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`numItemBlocks`.
"""
return self._set(numItemBlocks=value)
@since("1.4.0")
def setNumBlocks(self, value: int) -> "ALS":
"""
Sets both :py:attr:`numUserBlocks` and :py:attr:`numItemBlocks` to the specific value.
"""
self._set(numUserBlocks=value)
return self._set(numItemBlocks=value)
@since("1.4.0")
def setImplicitPrefs(self, value: bool) -> "ALS":
"""
Sets the value of :py:attr:`implicitPrefs`.
"""
return self._set(implicitPrefs=value)
@since("1.4.0")
def setAlpha(self, value: float) -> "ALS":
"""
Sets the value of :py:attr:`alpha`.
"""
return self._set(alpha=value)
@since("1.4.0")
def setUserCol(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`userCol`.
"""
return self._set(userCol=value)
@since("1.4.0")
def setItemCol(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`itemCol`.
"""
return self._set(itemCol=value)
@since("1.4.0")
def setRatingCol(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`ratingCol`.
"""
return self._set(ratingCol=value)
@since("1.4.0")
def setNonnegative(self, value: bool) -> "ALS":
"""
Sets the value of :py:attr:`nonnegative`.
"""
return self._set(nonnegative=value)
@since("2.0.0")
def setIntermediateStorageLevel(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`intermediateStorageLevel`.
"""
return self._set(intermediateStorageLevel=value)
@since("2.0.0")
def setFinalStorageLevel(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`finalStorageLevel`.
"""
return self._set(finalStorageLevel=value)
@since("2.2.0")
def setColdStartStrategy(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`coldStartStrategy`.
"""
return self._set(coldStartStrategy=value)
def setMaxIter(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`maxIter`.
"""
return self._set(maxIter=value)
def setRegParam(self, value: float) -> "ALS":
"""
Sets the value of :py:attr:`regParam`.
"""
return self._set(regParam=value)
def setPredictionCol(self, value: str) -> "ALS":
"""
Sets the value of :py:attr:`predictionCol`.
"""
return self._set(predictionCol=value)
def setCheckpointInterval(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`checkpointInterval`.
"""
return self._set(checkpointInterval=value)
def setSeed(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`seed`.
"""
return self._set(seed=value)
@since("3.0.0")
def setBlockSize(self, value: int) -> "ALS":
"""
Sets the value of :py:attr:`blockSize`.
"""
return self._set(blockSize=value)
class ALSModel(JavaModel, _ALSModelParams, JavaMLWritable, JavaMLReadable["ALSModel"]):
"""
Model fitted by ALS.
.. versionadded:: 1.4.0
"""
@since("3.0.0")
def setUserCol(self, value: str) -> "ALSModel":
"""
Sets the value of :py:attr:`userCol`.
"""
return self._set(userCol=value)
@since("3.0.0")
def setItemCol(self, value: str) -> "ALSModel":
"""
Sets the value of :py:attr:`itemCol`.
"""
return self._set(itemCol=value)
@since("3.0.0")
def setColdStartStrategy(self, value: str) -> "ALSModel":
"""
Sets the value of :py:attr:`coldStartStrategy`.
"""
return self._set(coldStartStrategy=value)
@since("3.0.0")
def setPredictionCol(self, value: str) -> "ALSModel":
"""
Sets the value of :py:attr:`predictionCol`.
"""
return self._set(predictionCol=value)
@since("3.0.0")
def setBlockSize(self, value: int) -> "ALSModel":
"""
Sets the value of :py:attr:`blockSize`.
"""
return self._set(blockSize=value)
@property # type: ignore[misc]
@since("1.4.0")
def rank(self) -> int:
"""rank of the matrix factorization model"""
return self._call_java("rank")
@property # type: ignore[misc]
@since("1.4.0")
def userFactors(self) -> DataFrame:
"""
a DataFrame that stores user factors in two columns: `id` and
`features`
"""
return self._call_java("userFactors")
@property # type: ignore[misc]
@since("1.4.0")
def itemFactors(self) -> DataFrame:
"""
a DataFrame that stores item factors in two columns: `id` and
`features`
"""
return self._call_java("itemFactors")
def recommendForAllUsers(self, numItems: int) -> DataFrame:
"""
Returns top `numItems` items recommended for each user, for all users.
.. versionadded:: 2.2.0
Parameters
----------
numItems : int
max number of recommendations for each user
Returns
-------
:py:class:`pyspark.sql.DataFrame`
a DataFrame of (userCol, recommendations), where recommendations are
stored as an array of (itemCol, rating) Rows.
"""
return self._call_java("recommendForAllUsers", numItems)
def recommendForAllItems(self, numUsers: int) -> DataFrame:
"""
Returns top `numUsers` users recommended for each item, for all items.
.. versionadded:: 2.2.0
Parameters
----------
numUsers : int
max number of recommendations for each item
Returns
-------
:py:class:`pyspark.sql.DataFrame`
a DataFrame of (itemCol, recommendations), where recommendations are
stored as an array of (userCol, rating) Rows.
"""
return self._call_java("recommendForAllItems", numUsers)
def recommendForUserSubset(self, dataset: DataFrame, numItems: int) -> DataFrame:
"""
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.
.. versionadded:: 2.3.0
Parameters
----------
dataset : :py:class:`pyspark.sql.DataFrame`
a DataFrame containing a column of user ids. The column name must match `userCol`.
numItems : int
max number of recommendations for each user
Returns
-------
:py:class:`pyspark.sql.DataFrame`
a DataFrame of (userCol, recommendations), where recommendations are
stored as an array of (itemCol, rating) Rows.
"""
return self._call_java("recommendForUserSubset", dataset, numItems)
def recommendForItemSubset(self, dataset: DataFrame, numUsers: int) -> DataFrame:
"""
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.
.. versionadded:: 2.3.0
Parameters
----------
dataset : :py:class:`pyspark.sql.DataFrame`
a DataFrame containing a column of item ids. The column name must match `itemCol`.
numUsers : int
max number of recommendations for each item
Returns
-------
:py:class:`pyspark.sql.DataFrame`
a DataFrame of (itemCol, recommendations), where recommendations are
stored as an array of (userCol, rating) Rows.
"""
return self._call_java("recommendForItemSubset", dataset, numUsers)
if __name__ == "__main__":
import doctest
import pyspark.ml.recommendation
from pyspark.sql import SparkSession
globs = pyspark.ml.recommendation.__dict__.copy()
# The small batch size here ensures that we see multiple batches,
# even in these small test examples:
spark = SparkSession.builder.master("local[2]").appName("ml.recommendation tests").getOrCreate()
sc = spark.sparkContext
globs["sc"] = sc
globs["spark"] = spark
import tempfile
temp_path = tempfile.mkdtemp()
globs["temp_path"] = temp_path
try:
(failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
spark.stop()
finally:
from shutil import rmtree
try:
rmtree(temp_path)
except OSError:
pass
if failure_count:
sys.exit(-1)