1717
1818package org .apache .spark .api .python
1919
20- import org .msgpack .ScalaMessagePack
2120import scala .util .Try
2221import org .apache .spark .rdd .RDD
23- import org .apache .spark .{SparkContext , Logging }
24- import org .apache .hadoop .io ._
22+ import org .apache .spark .Logging
2523import scala .util .Success
2624import scala .util .Failure
25+ import net .razorvine .pickle .Pickler
2726
2827/**
2928 * Utilities for serialization / deserialization between Python and Java, using MsgPack.
@@ -33,106 +32,58 @@ import scala.util.Failure
3332private [python] object SerDeUtil extends Logging {
3433
3534 /**
36- * Checks whether a Scala object needs to be registered with MsgPack. String, primitives
37- * and the standard collections don't need to be registered as MsgPack takes care of serializing
38- * them and registering them throws scary looking errors (but still works).
35+ * Convert an RDD of key-value pairs to an RDD of serialized Python objects, that is usable
36+ * by PySpark. By default, if serialization fails, toString is called and the string
37+ * representation is serialized
3938 */
40- def needsToBeRegistered [T ](t : T ) = {
41- t match {
42- case d : Double => false
43- case f : Float => false
44- case i : Int => false
45- case l : Long => false
46- case b : Byte => false
47- case c : Char => false
48- case bool : Boolean => false
49- case s : String => false
50- case m : Map [_, _] => false
51- case a : Seq [_] => false
52- case o : Option [_] => false
53- case _ => true
54- }
55- }
56-
57- /** Attempts to register a class with MsgPack */
58- def register [T ](t : T , msgpack : ScalaMessagePack ) {
59- if (! needsToBeRegistered(t)) {
60- return
61- }
62- val clazz = t.getClass
63- Try {
64- msgpack.register(clazz)
65- log.info(s " Registered key/value class with MsgPack: $clazz" )
66- } match {
67- case Failure (err) =>
68- log.warn(s """ Failed to register class ( $clazz) with MsgPack.
69- Falling back to default MsgPack serialization, or 'toString' as last resort.
70- Error: ${err.getMessage}""" )
71- case Success (result) =>
72- }
73- }
74-
75- /** Serializes an RDD[(K, V)] -> RDD[Array[Byte]] using MsgPack */
76- def serMsgPack [K , V ](rdd : RDD [(K , V )]) = {
77- import org .msgpack .ScalaMessagePack ._
78- rdd.mapPartitions{ pairs =>
79- val mp = new ScalaMessagePack
80- var triedReg = false
81- pairs.map{ pair =>
82- Try {
83- if (! triedReg) {
84- register(pair._1, mp)
85- register(pair._2, mp)
86- triedReg = true
39+ def rddToPython [K , V ](rdd : RDD [(K , V )]): RDD [Array [Byte ]] = {
40+ rdd.mapPartitions{ iter =>
41+ val pickle = new Pickler
42+ var keyFailed = false
43+ var valueFailed = false
44+ var firstRecord = true
45+ iter.map{ case (k, v) =>
46+ if (firstRecord) {
47+ Try {
48+ pickle.dumps(Array (k, v))
49+ } match {
50+ case Success (b) =>
51+ case Failure (err) =>
52+ val kt = Try {
53+ pickle.dumps(k)
54+ }
55+ val vt = Try {
56+ pickle.dumps(v)
57+ }
58+ (kt, vt) match {
59+ case (Failure (kf), Failure (vf)) =>
60+ log.warn(s """ Failed to pickle Java object as key: ${k.getClass.getSimpleName};
61+ Error: ${kf.getMessage}""" )
62+ log.warn(s """ Failed to pickle Java object as value: ${v.getClass.getSimpleName};
63+ Error: ${vf.getMessage}""" )
64+ keyFailed = true
65+ valueFailed = true
66+ case (Failure (kf), _) =>
67+ log.warn(s """ Failed to pickle Java object as key: ${k.getClass.getSimpleName};
68+ Error: ${kf.getMessage}""" )
69+ keyFailed = true
70+ case (_, Failure (vf)) =>
71+ log.warn(s """ Failed to pickle Java object as value: ${v.getClass.getSimpleName};
72+ Error: ${vf.getMessage}""" )
73+ valueFailed = true
74+ }
8775 }
88- mp.write(pair)
89- } match {
90- case Failure (err) =>
91- log.debug(" Failed to write" , err)
92- Try {
93- write((pair._1.toString, pair._2.toString))
94- } match {
95- case Success (result) => result
96- case Failure (e) => throw e
97- }
98- case Success (result) => result
76+ firstRecord = false
77+ }
78+ (keyFailed, valueFailed) match {
79+ case (true , true ) => pickle.dumps(Array (k.toString, v.toString))
80+ case (true , false ) => pickle.dumps(Array (k.toString, v))
81+ case (false , true ) => pickle.dumps(Array (k, v.toString))
82+ case (false , false ) => pickle.dumps(Array (k, v))
9983 }
10084 }
10185 }
10286 }
10387
104- /**
105- * Converts an RDD of key-value pairs, where key and/or value could be instances of
106- * [[org.apache.hadoop.io.Writable ]], into an RDD[(K, V)]
107- */
108- def convertRDD [K , V ](rdd : RDD [(K , V )]) = {
109- rdd.map{
110- case (k : Writable , v : Writable ) => (convert(k).asInstanceOf [K ], convert(v).asInstanceOf [V ])
111- case (k : Writable , v) => (convert(k).asInstanceOf [K ], v.asInstanceOf [V ])
112- case (k, v : Writable ) => (k.asInstanceOf [K ], convert(v).asInstanceOf [V ])
113- case (k, v) => (k.asInstanceOf [K ], v.asInstanceOf [V ])
114- }
115- }
116-
117- /** Converts a [[org.apache.hadoop.io.Writable ]] to the underlying primitive, String or
118- * object representation
119- */
120- def convert (writable : Writable ): Any = {
121- import collection .JavaConversions ._
122- writable match {
123- case iw : IntWritable => SparkContext .intWritableConverter().convert(iw)
124- case dw : DoubleWritable => SparkContext .doubleWritableConverter().convert(dw)
125- case lw : LongWritable => SparkContext .longWritableConverter().convert(lw)
126- case fw : FloatWritable => SparkContext .floatWritableConverter().convert(fw)
127- case t : Text => SparkContext .stringWritableConverter().convert(t)
128- case bw : BooleanWritable => SparkContext .booleanWritableConverter().convert(bw)
129- case byw : BytesWritable => SparkContext .bytesWritableConverter().convert(byw)
130- case n : NullWritable => None
131- case aw : ArrayWritable => aw.get().map(convert(_))
132- case mw : MapWritable => mw.map{ case (k, v) => (convert(k), convert(v)) }.toMap
133- case other => other
134- }
135- }
136-
13788}
13889
0 commit comments