@@ -22,12 +22,10 @@ import java.net._
22
22
import java .util .{List => JList , ArrayList => JArrayList , Map => JMap , Collections }
23
23
24
24
import scala .collection .JavaConversions ._
25
- import scala .collection .JavaConverters ._
26
25
import scala .collection .mutable
27
26
import scala .language .existentials
28
27
29
28
import com .google .common .base .Charsets .UTF_8
30
- import net .razorvine .pickle .{Pickler , Unpickler }
31
29
32
30
import org .apache .hadoop .conf .Configuration
33
31
import org .apache .hadoop .io .compress .CompressionCodec
@@ -442,7 +440,7 @@ private[spark] object PythonRDD extends Logging {
442
440
val rdd = sc.sc.sequenceFile[K , V ](path, kc, vc, minSplits)
443
441
val confBroadcasted = sc.sc.broadcast(new SerializableWritable (sc.hadoopConfiguration()))
444
442
val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
445
- new WritableToJavaConverter (confBroadcasted, batchSize ))
443
+ new WritableToJavaConverter (confBroadcasted))
446
444
JavaRDD .fromRDD(SerDeUtil .pairRDDToPython(converted, batchSize))
447
445
}
448
446
@@ -468,7 +466,7 @@ private[spark] object PythonRDD extends Logging {
468
466
Some (path), inputFormatClass, keyClass, valueClass, mergedConf)
469
467
val confBroadcasted = sc.sc.broadcast(new SerializableWritable (mergedConf))
470
468
val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
471
- new WritableToJavaConverter (confBroadcasted, batchSize ))
469
+ new WritableToJavaConverter (confBroadcasted))
472
470
JavaRDD .fromRDD(SerDeUtil .pairRDDToPython(converted, batchSize))
473
471
}
474
472
@@ -494,7 +492,7 @@ private[spark] object PythonRDD extends Logging {
494
492
None , inputFormatClass, keyClass, valueClass, conf)
495
493
val confBroadcasted = sc.sc.broadcast(new SerializableWritable (conf))
496
494
val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
497
- new WritableToJavaConverter (confBroadcasted, batchSize ))
495
+ new WritableToJavaConverter (confBroadcasted))
498
496
JavaRDD .fromRDD(SerDeUtil .pairRDDToPython(converted, batchSize))
499
497
}
500
498
@@ -537,7 +535,7 @@ private[spark] object PythonRDD extends Logging {
537
535
Some (path), inputFormatClass, keyClass, valueClass, mergedConf)
538
536
val confBroadcasted = sc.sc.broadcast(new SerializableWritable (mergedConf))
539
537
val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
540
- new WritableToJavaConverter (confBroadcasted, batchSize ))
538
+ new WritableToJavaConverter (confBroadcasted))
541
539
JavaRDD .fromRDD(SerDeUtil .pairRDDToPython(converted, batchSize))
542
540
}
543
541
@@ -563,7 +561,7 @@ private[spark] object PythonRDD extends Logging {
563
561
None , inputFormatClass, keyClass, valueClass, conf)
564
562
val confBroadcasted = sc.sc.broadcast(new SerializableWritable (conf))
565
563
val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
566
- new WritableToJavaConverter (confBroadcasted, batchSize ))
564
+ new WritableToJavaConverter (confBroadcasted))
567
565
JavaRDD .fromRDD(SerDeUtil .pairRDDToPython(converted, batchSize))
568
566
}
569
567
@@ -746,104 +744,6 @@ private[spark] object PythonRDD extends Logging {
746
744
converted.saveAsHadoopDataset(new JobConf (conf))
747
745
}
748
746
}
749
-
750
-
751
- /**
752
- * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions).
753
- */
754
- @ deprecated(" PySpark does not use it anymore" , " 1.1" )
755
- def pythonToJavaMap (pyRDD : JavaRDD [Array [Byte ]]): JavaRDD [Map [String , _]] = {
756
- pyRDD.rdd.mapPartitions { iter =>
757
- val unpickle = new Unpickler
758
- SerDeUtil .initialize()
759
- iter.flatMap { row =>
760
- unpickle.loads(row) match {
761
- // in case of objects are pickled in batch mode
762
- case objs : JArrayList [JMap [String , _] @ unchecked] => objs.map(_.toMap)
763
- // not in batch mode
764
- case obj : JMap [String @ unchecked, _] => Seq (obj.toMap)
765
- }
766
- }
767
- }
768
- }
769
-
770
- /**
771
- * Convert an RDD of serialized Python tuple to Array (no recursive conversions).
772
- * It is only used by pyspark.sql.
773
- */
774
- def pythonToJavaArray (pyRDD : JavaRDD [Array [Byte ]], batched : Boolean ): JavaRDD [Array [_]] = {
775
-
776
- def toArray (obj : Any ): Array [_] = {
777
- obj match {
778
- case objs : JArrayList [_] =>
779
- objs.toArray
780
- case obj if obj.getClass.isArray =>
781
- obj.asInstanceOf [Array [_]].toArray
782
- }
783
- }
784
-
785
- pyRDD.rdd.mapPartitions { iter =>
786
- val unpickle = new Unpickler
787
- iter.flatMap { row =>
788
- val obj = unpickle.loads(row)
789
- if (batched) {
790
- obj.asInstanceOf [JArrayList [_]].map(toArray)
791
- } else {
792
- Seq (toArray(obj))
793
- }
794
- }
795
- }.toJavaRDD()
796
- }
797
-
798
- private [spark] class AutoBatchedPickler (iter : Iterator [Any ]) extends Iterator [Array [Byte ]] {
799
- private val pickle = new Pickler ()
800
- private var batch = 1
801
- private val buffer = new mutable.ArrayBuffer [Any ]
802
-
803
- override def hasNext (): Boolean = iter.hasNext
804
-
805
- override def next (): Array [Byte ] = {
806
- while (iter.hasNext && buffer.length < batch) {
807
- buffer += iter.next()
808
- }
809
- val bytes = pickle.dumps(buffer.toArray)
810
- val size = bytes.length
811
- // let 1M < size < 10M
812
- if (size < 1024 * 1024 ) {
813
- batch *= 2
814
- } else if (size > 1024 * 1024 * 10 && batch > 1 ) {
815
- batch /= 2
816
- }
817
- buffer.clear()
818
- bytes
819
- }
820
- }
821
-
822
- /**
823
- * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by
824
- * PySpark.
825
- */
826
- def javaToPython (jRDD : JavaRDD [Any ]): JavaRDD [Array [Byte ]] = {
827
- jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler (iter) }
828
- }
829
-
830
- /**
831
- * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark.
832
- */
833
- def pythonToJava (pyRDD : JavaRDD [Array [Byte ]], batched : Boolean ): JavaRDD [Any ] = {
834
- pyRDD.rdd.mapPartitions { iter =>
835
- SerDeUtil .initialize()
836
- val unpickle = new Unpickler
837
- iter.flatMap { row =>
838
- val obj = unpickle.loads(row)
839
- if (batched) {
840
- obj.asInstanceOf [JArrayList [_]].asScala
841
- } else {
842
- Seq (obj)
843
- }
844
- }
845
- }.toJavaRDD()
846
- }
847
747
}
848
748
849
749
private
0 commit comments