@@ -450,19 +450,19 @@ setMethod("combineByKey",
450
450
})
451
451
452
452
# ' Aggregate a pair RDD by each key.
453
- # '
453
+ # '
454
454
# ' Aggregate the values of each key in an RDD, using given combine functions
455
455
# ' and a neutral "zero value". This function can return a different result type,
456
456
# ' U, than the type of the values in this RDD, V. Thus, we need one operation
457
- # ' for merging a V into a U and one operation for merging two U's, The former
458
- # ' operation is used for merging values within a partition, and the latter is
459
- # ' used for merging values between partitions. To avoid memory allocation, both
460
- # ' of these functions are allowed to modify and return their first argument
457
+ # ' for merging a V into a U and one operation for merging two U's, The former
458
+ # ' operation is used for merging values within a partition, and the latter is
459
+ # ' used for merging values between partitions. To avoid memory allocation, both
460
+ # ' of these functions are allowed to modify and return their first argument
461
461
# ' instead of creating a new U.
462
- # '
462
+ # '
463
463
# ' @param x An RDD.
464
464
# ' @param zeroValue A neutral "zero value".
465
- # ' @param seqOp A function to aggregate the values of each key. It may return
465
+ # ' @param seqOp A function to aggregate the values of each key. It may return
466
466
# ' a different result type from the type of the values.
467
467
# ' @param combOp A function to aggregate results of seqOp.
468
468
# ' @return An RDD containing the aggregation result.
@@ -474,7 +474,7 @@ setMethod("combineByKey",
474
474
# ' zeroValue <- list(0, 0)
475
475
# ' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) }
476
476
# ' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) }
477
- # ' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L)
477
+ # ' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L)
478
478
# ' # list(list(1, list(3, 2)), list(2, list(7, 2)))
479
479
# '}
480
480
# ' @rdname aggregateByKey
@@ -491,12 +491,12 @@ setMethod("aggregateByKey",
491
491
})
492
492
493
493
# ' Fold a pair RDD by each key.
494
- # '
494
+ # '
495
495
# ' Aggregate the values of each key in an RDD, using an associative function "func"
496
- # ' and a neutral "zero value" which may be added to the result an arbitrary
497
- # ' number of times, and must not change the result (e.g., 0 for addition, or
496
+ # ' and a neutral "zero value" which may be added to the result an arbitrary
497
+ # ' number of times, and must not change the result (e.g., 0 for addition, or
498
498
# ' 1 for multiplication.).
499
- # '
499
+ # '
500
500
# ' @param x An RDD.
501
501
# ' @param zeroValue A neutral "zero value".
502
502
# ' @param func An associative function for folding values of each key.
@@ -546,11 +546,11 @@ setMethod("join",
546
546
function (x , y , numPartitions ) {
547
547
xTagged <- lapply(x , function (i ) { list (i [[1 ]], list (1L , i [[2 ]])) })
548
548
yTagged <- lapply(y , function (i ) { list (i [[1 ]], list (2L , i [[2 ]])) })
549
-
549
+
550
550
doJoin <- function (v ) {
551
551
joinTaggedList(v , list (FALSE , FALSE ))
552
552
}
553
-
553
+
554
554
joined <- flatMapValues(groupByKey(unionRDD(xTagged , yTagged ), numToInt(numPartitions )),
555
555
doJoin )
556
556
})
@@ -566,8 +566,8 @@ setMethod("join",
566
566
# ' @param y An RDD to be joined. Should be an RDD where each element is
567
567
# ' list(K, V).
568
568
# ' @param numPartitions Number of partitions to create.
569
- # ' @return For each element (k, v) in x, the resulting RDD will either contain
570
- # ' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL))
569
+ # ' @return For each element (k, v) in x, the resulting RDD will either contain
570
+ # ' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL))
571
571
# ' if no elements in rdd2 have key k.
572
572
# ' @examples
573
573
# '\dontrun{
@@ -584,11 +584,11 @@ setMethod("leftOuterJoin",
584
584
function (x , y , numPartitions ) {
585
585
xTagged <- lapply(x , function (i ) { list (i [[1 ]], list (1L , i [[2 ]])) })
586
586
yTagged <- lapply(y , function (i ) { list (i [[1 ]], list (2L , i [[2 ]])) })
587
-
587
+
588
588
doJoin <- function (v ) {
589
589
joinTaggedList(v , list (FALSE , TRUE ))
590
590
}
591
-
591
+
592
592
joined <- flatMapValues(groupByKey(unionRDD(xTagged , yTagged ), numPartitions ), doJoin )
593
593
})
594
594
@@ -621,18 +621,18 @@ setMethod("rightOuterJoin",
621
621
function (x , y , numPartitions ) {
622
622
xTagged <- lapply(x , function (i ) { list (i [[1 ]], list (1L , i [[2 ]])) })
623
623
yTagged <- lapply(y , function (i ) { list (i [[1 ]], list (2L , i [[2 ]])) })
624
-
624
+
625
625
doJoin <- function (v ) {
626
626
joinTaggedList(v , list (TRUE , FALSE ))
627
627
}
628
-
628
+
629
629
joined <- flatMapValues(groupByKey(unionRDD(xTagged , yTagged ), numPartitions ), doJoin )
630
630
})
631
631
632
632
# ' Full outer join two RDDs
633
633
# '
634
634
# ' @description
635
- # ' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V).
635
+ # ' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V).
636
636
# ' The key types of the two RDDs should be the same.
637
637
# '
638
638
# ' @param x An RDD to be joined. Should be an RDD where each element is
@@ -642,7 +642,7 @@ setMethod("rightOuterJoin",
642
642
# ' @param numPartitions Number of partitions to create.
643
643
# ' @return For each element (k, v) in x and (k, w) in y, the resulting RDD
644
644
# ' will contain all pairs (k, (v, w)) for both (k, v) in x and
645
- # ' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements
645
+ # ' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements
646
646
# ' in x/y have key k.
647
647
# ' @examples
648
648
# '\dontrun{
@@ -681,7 +681,7 @@ setMethod("fullOuterJoin",
681
681
# ' sc <- sparkR.init()
682
682
# ' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4)))
683
683
# ' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3)))
684
- # ' cogroup(rdd1, rdd2, numPartitions = 2L)
684
+ # ' cogroup(rdd1, rdd2, numPartitions = 2L)
685
685
# ' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list()))
686
686
# '}
687
687
# ' @rdname cogroup
@@ -692,7 +692,7 @@ setMethod("cogroup",
692
692
rdds <- list (... )
693
693
rddsLen <- length(rdds )
694
694
for (i in 1 : rddsLen ) {
695
- rdds [[i ]] <- lapply(rdds [[i ]],
695
+ rdds [[i ]] <- lapply(rdds [[i ]],
696
696
function (x ) { list (x [[1 ]], list (i , x [[2 ]])) })
697
697
# TODO(hao): As issue [SparkR-142] mentions, the right value of i
698
698
# will not be captured into UDF if getJRDD is not invoked.
@@ -721,7 +721,7 @@ setMethod("cogroup",
721
721
}
722
722
})
723
723
}
724
- cogroup.rdd <- mapValues(groupByKey(union.rdd , numPartitions ),
724
+ cogroup.rdd <- mapValues(groupByKey(union.rdd , numPartitions ),
725
725
group.func )
726
726
})
727
727
@@ -743,18 +743,18 @@ setMethod("sortByKey",
743
743
signature(x = " RDD" ),
744
744
function (x , ascending = TRUE , numPartitions = SparkR :: numPartitions(x )) {
745
745
rangeBounds <- list ()
746
-
746
+
747
747
if (numPartitions > 1 ) {
748
748
rddSize <- count(x )
749
749
# constant from Spark's RangePartitioner
750
750
maxSampleSize <- numPartitions * 20
751
751
fraction <- min(maxSampleSize / max(rddSize , 1 ), 1.0 )
752
-
752
+
753
753
samples <- collect(keys(sampleRDD(x , FALSE , fraction , 1L )))
754
-
754
+
755
755
# Note: the built-in R sort() function only works on atomic vectors
756
756
samples <- sort(unlist(samples , recursive = FALSE ), decreasing = ! ascending )
757
-
757
+
758
758
if (length(samples ) > 0 ) {
759
759
rangeBounds <- lapply(seq_len(numPartitions - 1 ),
760
760
function (i ) {
@@ -766,24 +766,109 @@ setMethod("sortByKey",
766
766
767
767
rangePartitionFunc <- function (key ) {
768
768
partition <- 0
769
-
769
+
770
770
# TODO: Use binary search instead of linear search, similar with Spark
771
771
while (partition < length(rangeBounds ) && key > rangeBounds [[partition + 1 ]]) {
772
772
partition <- partition + 1
773
773
}
774
-
774
+
775
775
if (ascending ) {
776
776
partition
777
777
} else {
778
778
numPartitions - partition - 1
779
779
}
780
780
}
781
-
781
+
782
782
partitionFunc <- function (part ) {
783
783
sortKeyValueList(part , decreasing = ! ascending )
784
784
}
785
-
785
+
786
786
newRDD <- partitionBy(x , numPartitions , rangePartitionFunc )
787
787
lapplyPartition(newRDD , partitionFunc )
788
788
})
789
-
789
+
790
+ # ' @description
791
+ # ' \code{sampleByKey} return a subset RDD of the given RDD sampled by key
792
+ # '
793
+ # ' @param x The RDD to sample elements by key, where each element is
794
+ # ' list(K, V) or c(K, V).
795
+ # ' @param withReplacement Sampling with replacement or not
796
+ # ' @param fraction The (rough) sample target fraction
797
+ # ' @param seed Randomness seed value
798
+ # ' @examples
799
+ # '\dontrun{
800
+ # ' sc <- sparkR.init()
801
+ # ' rdd <- parallelize(sc, 1:3000)
802
+ # ' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x)
803
+ # ' else { if (x %% 3 == 1) list("b", x) else list("c", x) }})
804
+ # ' fractions <- list(a = 0.2, b = 0.1, c = 0.3)
805
+ # ' sample <- sampleByKey(pairs, FALSE, fractions, 1618L)
806
+ # ' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE
807
+ # ' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE
808
+ # ' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE
809
+ # ' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE
810
+ # ' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE
811
+ # ' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE
812
+ # ' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE
813
+ # ' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE
814
+ # ' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE
815
+ # ' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4)
816
+ # ' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored
817
+ # ' fractions <- list(a = 0.2, b = 0.1)
818
+ # ' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c"
819
+ # '}
820
+ # ' @rdname sampleByKey
821
+ # ' @aliases sampleByKey,RDD-method
822
+ setMethod ("sampleByKey ",
823
+ signature(x = " RDD" , withReplacement = " logical" ,
824
+ fractions = " vector" , seed = " integer" ),
825
+ function (x , withReplacement , fractions , seed ) {
826
+
827
+ for (elem in fractions ) {
828
+ if (elem < 0.0 ) {
829
+ stop(paste(" Negative fraction value " , fractions [which(fractions == elem )]))
830
+ }
831
+ }
832
+
833
+ # The sampler: takes a partition and returns its sampled version.
834
+ samplingFunc <- function (split , part ) {
835
+ set.seed(bitwXor(seed , split ))
836
+ res <- vector(" list" , length(part ))
837
+ len <- 0
838
+
839
+ # mixing because the initial seeds are close to each other
840
+ runif(10 )
841
+
842
+ for (elem in part ) {
843
+ if (elem [[1 ]] %in% names(fractions )) {
844
+ frac <- as.numeric(fractions [which(elem [[1 ]] == names(fractions ))])
845
+ if (withReplacement ) {
846
+ count <- rpois(1 , frac )
847
+ if (count > 0 ) {
848
+ res [(len + 1 ): (len + count )] <- rep(list (elem ), count )
849
+ len <- len + count
850
+ }
851
+ } else {
852
+ if (runif(1 ) < frac ) {
853
+ len <- len + 1
854
+ res [[len ]] <- elem
855
+ }
856
+ }
857
+ } else {
858
+ stop(" KeyError: \" " , elem [[1 ]], " \" " )
859
+ }
860
+ }
861
+
862
+ # TODO(zongheng): look into the performance of the current
863
+ # implementation. Look into some iterator package? Note that
864
+ # Scala avoids many calls to creating an empty list and PySpark
865
+ # similarly achieves this using `yield'. (duplicated from sampleRDD)
866
+ if (len > 0 ) {
867
+ res [1 : len ]
868
+ } else {
869
+ list ()
870
+ }
871
+ }
872
+
873
+ lapplyPartitionsWithIndex(x , samplingFunc )
874
+ })
0 commit comments