|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.api.python |
| 19 | + |
| 20 | +import org.apache.spark.rdd.RDD |
| 21 | +import org.apache.spark.Logging |
| 22 | +import org.apache.hadoop.conf.Configuration |
| 23 | +import org.apache.hadoop.io._ |
| 24 | +import scala.util.{Failure, Success, Try} |
| 25 | +import org.apache.spark.annotation.Experimental |
| 26 | + |
| 27 | + |
| 28 | +/** |
| 29 | + * :: Experimental :: |
| 30 | + * A trait for use with reading custom classes in PySpark. Implement this trait and add custom |
| 31 | + * transformation code by overriding the convert method. |
| 32 | + */ |
| 33 | +@Experimental |
| 34 | +trait Converter[T, U] extends Serializable { |
| 35 | + def convert(obj: T): U |
| 36 | +} |
| 37 | + |
| 38 | +private[python] object Converter extends Logging { |
| 39 | + |
| 40 | + def getInstance(converterClass: Option[String]): Converter[Any, Any] = { |
| 41 | + converterClass.map { cc => |
| 42 | + Try { |
| 43 | + val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]] |
| 44 | + logInfo(s"Loaded converter: $cc") |
| 45 | + c |
| 46 | + } match { |
| 47 | + case Success(c) => c |
| 48 | + case Failure(err) => |
| 49 | + logError(s"Failed to load converter: $cc") |
| 50 | + throw err |
| 51 | + } |
| 52 | + }.getOrElse { new DefaultConverter } |
| 53 | + } |
| 54 | +} |
| 55 | + |
| 56 | +/** |
| 57 | + * A converter that handles conversion of common [[org.apache.hadoop.io.Writable]] objects. |
| 58 | + * Other objects are passed through without conversion. |
| 59 | + */ |
| 60 | +private[python] class DefaultConverter extends Converter[Any, Any] { |
| 61 | + |
| 62 | + /** |
| 63 | + * Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or |
| 64 | + * object representation |
| 65 | + */ |
| 66 | + private def convertWritable(writable: Writable): Any = { |
| 67 | + import collection.JavaConversions._ |
| 68 | + writable match { |
| 69 | + case iw: IntWritable => iw.get() |
| 70 | + case dw: DoubleWritable => dw.get() |
| 71 | + case lw: LongWritable => lw.get() |
| 72 | + case fw: FloatWritable => fw.get() |
| 73 | + case t: Text => t.toString |
| 74 | + case bw: BooleanWritable => bw.get() |
| 75 | + case byw: BytesWritable => byw.getBytes |
| 76 | + case n: NullWritable => null |
| 77 | + case aw: ArrayWritable => aw.get().map(convertWritable(_)) |
| 78 | + case mw: MapWritable => mapAsJavaMap(mw.map { case (k, v) => |
| 79 | + (convertWritable(k), convertWritable(v)) |
| 80 | + }.toMap) |
| 81 | + case other => other |
| 82 | + } |
| 83 | + } |
| 84 | + |
| 85 | + def convert(obj: Any): Any = { |
| 86 | + obj match { |
| 87 | + case writable: Writable => |
| 88 | + convertWritable(writable) |
| 89 | + case _ => |
| 90 | + obj |
| 91 | + } |
| 92 | + } |
| 93 | +} |
| 94 | + |
| 95 | +/** Utilities for working with Python objects <-> Hadoop-related objects */ |
| 96 | +private[python] object PythonHadoopUtil { |
| 97 | + |
| 98 | + /** |
| 99 | + * Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]] |
| 100 | + */ |
| 101 | + def mapToConf(map: java.util.Map[String, String]): Configuration = { |
| 102 | + import collection.JavaConversions._ |
| 103 | + val conf = new Configuration() |
| 104 | + map.foreach{ case (k, v) => conf.set(k, v) } |
| 105 | + conf |
| 106 | + } |
| 107 | + |
| 108 | + /** |
| 109 | + * Merges two configurations, returns a copy of left with keys from right overwriting |
| 110 | + * any matching keys in left |
| 111 | + */ |
| 112 | + def mergeConfs(left: Configuration, right: Configuration): Configuration = { |
| 113 | + import collection.JavaConversions._ |
| 114 | + val copy = new Configuration(left) |
| 115 | + right.iterator().foreach(entry => copy.set(entry.getKey, entry.getValue)) |
| 116 | + copy |
| 117 | + } |
| 118 | + |
| 119 | + /** |
| 120 | + * Converts an RDD of key-value pairs, where key and/or value could be instances of |
| 121 | + * [[org.apache.hadoop.io.Writable]], into an RDD[(K, V)] |
| 122 | + */ |
| 123 | + def convertRDD[K, V](rdd: RDD[(K, V)], |
| 124 | + keyConverter: Converter[Any, Any], |
| 125 | + valueConverter: Converter[Any, Any]): RDD[(Any, Any)] = { |
| 126 | + rdd.map { case (k, v) => (keyConverter.convert(k), valueConverter.convert(v)) } |
| 127 | + } |
| 128 | + |
| 129 | +} |
0 commit comments