diff options
author | Uri Laserson <laserson@cloudera.com> | 2014-09-27 21:48:05 -0700 |
---|---|---|
committer | Matei Zaharia <matei@databricks.com> | 2014-09-27 21:48:05 -0700 |
commit | 248232936e1bead7f102e59eb8faf3126c582d9d (patch) | |
tree | 8bee3f6fd2ba42230777a5a10ed47deb8f6863b1 /examples/src/main/scala | |
parent | 5b922bb458e863f5be0ae68167de882743f70b86 (diff) | |
download | spark-248232936e1bead7f102e59eb8faf3126c582d9d.tar.gz spark-248232936e1bead7f102e59eb8faf3126c582d9d.tar.bz2 spark-248232936e1bead7f102e59eb8faf3126c582d9d.zip |
[SPARK-3389] Add Converter for ease of Parquet reading in PySpark
https://issues.apache.org/jira/browse/SPARK-3389
Author: Uri Laserson <laserson@cloudera.com>
Closes #2256 from laserson/SPARK-3389 and squashes the following commits:
0ed363e [Uri Laserson] PEP8'd the python file
0b4b380 [Uri Laserson] Moved converter to examples and added python example
eecf4dc [Uri Laserson] [SPARK-3389] Add Converter for ease of Parquet reading in PySpark
Diffstat (limited to 'examples/src/main/scala')
-rw-r--r-- | examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala | 76 |
1 files changed, 46 insertions, 30 deletions
diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala index 1b25983a38..a11890d6f2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -30,21 +30,28 @@ import org.apache.spark.api.python.Converter import org.apache.spark.SparkException -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts - * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries - * to work with all 3 Avro data mappings (Generic, Specific and Reflect). - */ -class AvroWrapperToJavaConverter extends Converter[Any, Any] { - override def convert(obj: Any): Any = { +object AvroConversionUtil extends Serializable { + def fromAvro(obj: Any, schema: Schema): Any = { if (obj == null) { return null } - obj.asInstanceOf[AvroWrapper[_]].datum() match { - case null => null - case record: IndexedRecord => unpackRecord(record) - case other => throw new SparkException( - s"Unsupported top-level Avro data type ${other.getClass.getName}") + schema.getType match { + case UNION => unpackUnion(obj, schema) + case ARRAY => unpackArray(obj, schema) + case FIXED => unpackFixed(obj, schema) + case MAP => unpackMap(obj, schema) + case BYTES => unpackBytes(obj) + case RECORD => unpackRecord(obj) + case STRING => obj.toString + case ENUM => obj.toString + case NULL => obj + case BOOLEAN => obj + case DOUBLE => obj + case FLOAT => obj + case INT => obj + case LONG => obj + case other => throw new SparkException( + s"Unknown Avro schema type ${other.getName}") } } @@ -103,28 +110,37 @@ class AvroWrapperToJavaConverter extends Converter[Any, Any] { "Unions may only consist of a concrete type and null") } } +} - def fromAvro(obj: Any, schema: Schema): Any = { +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro IndexedRecord (e.g., derived from AvroParquetInputFormat) to a Java Map. + */ +class IndexedRecordToJavaConverter extends Converter[IndexedRecord, JMap[String, Any]]{ + override def convert(record: IndexedRecord): JMap[String, Any] = { + if (record == null) { + return null + } + val map = new java.util.HashMap[String, Any] + AvroConversionUtil.unpackRecord(record) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries + * to work with all 3 Avro data mappings (Generic, Specific and Reflect). + */ +class AvroWrapperToJavaConverter extends Converter[Any, Any] { + override def convert(obj: Any): Any = { if (obj == null) { return null } - schema.getType match { - case UNION => unpackUnion(obj, schema) - case ARRAY => unpackArray(obj, schema) - case FIXED => unpackFixed(obj, schema) - case MAP => unpackMap(obj, schema) - case BYTES => unpackBytes(obj) - case RECORD => unpackRecord(obj) - case STRING => obj.toString - case ENUM => obj.toString - case NULL => obj - case BOOLEAN => obj - case DOUBLE => obj - case FLOAT => obj - case INT => obj - case LONG => obj - case other => throw new SparkException( - s"Unknown Avro schema type ${other.getName}") + obj.asInstanceOf[AvroWrapper[_]].datum() match { + case null => null + case record: IndexedRecord => AvroConversionUtil.unpackRecord(record) + case other => throw new SparkException( + s"Unsupported top-level Avro data type ${other.getClass.getName}") } } } |