diff options
author | Kan Zhang <kzhang@apache.org> | 2014-07-30 13:19:05 -0700 |
---|---|---|
committer | Josh Rosen <joshrosen@apache.org> | 2014-07-30 13:19:05 -0700 |
commit | 94d1f46fc43c0cb85125f757fb40db9271caf1f4 (patch) | |
tree | 8878443a963ad6ce5ba3af679567d893c8df70cc /examples/src/main/scala | |
parent | 437dc8c5b54f0dcf9564c1fb07e8dce9e771c8cd (diff) | |
download | spark-94d1f46fc43c0cb85125f757fb40db9271caf1f4.tar.gz spark-94d1f46fc43c0cb85125f757fb40db9271caf1f4.tar.bz2 spark-94d1f46fc43c0cb85125f757fb40db9271caf1f4.zip |
[SPARK-2024] Add saveAsSequenceFile to PySpark
JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024
This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats.
* Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs.
* Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types.
* No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples.
* Added HBase and Cassandra output examples to show how custom output formats and converters can be used.
cc MLnick mateiz ahirreddy pwendell
Author: Kan Zhang <kzhang@apache.org>
Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits:
c01e3ef [Kan Zhang] [SPARK-2024] code formatting
6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD
d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10
57a7a5e [Kan Zhang] [SPARK-2024] correcting typo
75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD
0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests
9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests
0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases
7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
Diffstat (limited to 'examples/src/main/scala')
3 files changed, 93 insertions, 34 deletions
diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala index 29a65c7a5f..83feb5703b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.pythonconverters import org.apache.spark.api.python.Converter import java.nio.ByteBuffer import org.apache.cassandra.utils.ByteBufferUtil -import collection.JavaConversions.{mapAsJavaMap, mapAsScalaMap} +import collection.JavaConversions._ /** @@ -44,3 +44,25 @@ class CassandraCQLValueConverter extends Converter[Any, java.util.Map[String, St mapAsJavaMap(result.mapValues(bb => ByteBufferUtil.string(bb))) } } + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * Map[String, Int] to Cassandra key + */ +class ToCassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, ByteBuffer]] { + override def convert(obj: Any): java.util.Map[String, ByteBuffer] = { + val input = obj.asInstanceOf[java.util.Map[String, Int]] + mapAsJavaMap(input.mapValues(i => ByteBufferUtil.bytes(i))) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * List[String] to Cassandra value + */ +class ToCassandraCQLValueConverter extends Converter[Any, java.util.List[ByteBuffer]] { + override def convert(obj: Any): java.util.List[ByteBuffer] = { + val input = obj.asInstanceOf[java.util.List[String]] + seqAsJavaList(input.map(s => ByteBufferUtil.bytes(s))) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala deleted file mode 100644 index 42ae960bd6..0000000000 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.pythonconverters - -import org.apache.spark.api.python.Converter -import org.apache.hadoop.hbase.client.Result -import org.apache.hadoop.hbase.util.Bytes - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a HBase Result - * to a String - */ -class HBaseConverter extends Converter[Any, String] { - override def convert(obj: Any): String = { - val result = obj.asInstanceOf[Result] - Bytes.toStringBinary(result.value()) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala new file mode 100644 index 0000000000..273bee0a8b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.pythonconverters + +import scala.collection.JavaConversions._ + +import org.apache.spark.api.python.Converter +import org.apache.hadoop.hbase.client.{Put, Result} +import org.apache.hadoop.hbase.io.ImmutableBytesWritable +import org.apache.hadoop.hbase.util.Bytes + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts an + * HBase Result to a String + */ +class HBaseResultToStringConverter extends Converter[Any, String] { + override def convert(obj: Any): String = { + val result = obj.asInstanceOf[Result] + Bytes.toStringBinary(result.value()) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts an + * ImmutableBytesWritable to a String + */ +class ImmutableBytesWritableToStringConverter extends Converter[Any, String] { + override def convert(obj: Any): String = { + val key = obj.asInstanceOf[ImmutableBytesWritable] + Bytes.toStringBinary(key.get()) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * String to an ImmutableBytesWritable + */ +class StringToImmutableBytesWritableConverter extends Converter[Any, ImmutableBytesWritable] { + override def convert(obj: Any): ImmutableBytesWritable = { + val bytes = Bytes.toBytes(obj.asInstanceOf[String]) + new ImmutableBytesWritable(bytes) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * list of Strings to HBase Put + */ +class StringListToPutConverter extends Converter[Any, Put] { + override def convert(obj: Any): Put = { + val output = obj.asInstanceOf[java.util.ArrayList[String]].map(Bytes.toBytes(_)).toArray + val put = new Put(output(0)) + put.add(output(1), output(2), output(3)) + } +} |