Davies,

That’s pretty neat. I heard there was a pure Python clone of Spark out
there—so you were one of the people behind it!

I’ve created a JIRA issue about this. SPARK-3533: Add saveAsTextFileByKey()
method to RDDs <https://issues.apache.org/jira/browse/SPARK-3533>

Sean,

I think you might be able to get this working with a subclass of
MultipleTextOutputFormat, which overrides generateFileNameForKeyValue,
generateActualKey, etc. A bit of work for sure, but probably works.

I’m looking at how to make this work in PySpark as of 1.1.0. The closest
examples I can see of how to use the saveAsHadoop...() methods in this way
are these two examples: HBase Output Format
<https://github.com/apache/spark/blob/cc14644460872efb344e8d895859d70213a40840/examples/src/main/python/hbase_outputformat.py#L60>
and Avro Input Format
<https://github.com/apache/spark/blob/cc14644460872efb344e8d895859d70213a40840/examples/src/main/python/avro_inputformat.py#L73>

Basically, I’m thinking I need to subclass MultipleTextOutputFormat and
override some methods in a Scala file, and then reference that from Python?
Like how the AvroWrapperToJavaConverter class is done? Seems pretty
involved, but I’ll give it a shot if that’s the right direction to go in.

Nick
​

On Mon, Sep 15, 2014 at 1:08 PM, Davies Liu <dav...@databricks.com> wrote:

> Maybe we should provide an API like saveTextFilesByKey(path),
> could you create an JIRA for it ?
>
> There is one in DPark [1] actually.
>
> [1] https://github.com/douban/dpark/blob/master/dpark/rdd.py#L309
>
> On Mon, Sep 15, 2014 at 7:08 AM, Nicholas Chammas
> <nicholas.cham...@gmail.com> wrote:
> > Any tips from anybody on how to do this in PySpark? (Or regular Spark,
> for
> > that matter.)
> >
> > On Sat, Sep 13, 2014 at 1:25 PM, Nick Chammas <
> nicholas.cham...@gmail.com>
> > wrote:
> >>
> >> Howdy doody Spark Users,
> >>
> >> I’d like to somehow write out a single RDD to multiple paths in one go.
> >> Here’s an example.
> >>
> >> I have an RDD of (key, value) pairs like this:
> >>
> >> >>> a = sc.parallelize(['Nick', 'Nancy', 'Bob', 'Ben',
> >> >>> 'Frankie']).keyBy(lambda x: x[0])
> >> >>> a.collect()
> >> [('N', 'Nick'), ('N', 'Nancy'), ('B', 'Bob'), ('B', 'Ben'), ('F',
> >> 'Frankie')]
> >>
> >> Now I want to write the RDD out to different paths depending on the
> keys,
> >> so that I have one output directory per distinct key. Each output
> directory
> >> could potentially have multiple part- files or whatever.
> >>
> >> So my output would be something like:
> >>
> >> /path/prefix/n [/part-1, /part-2, etc]
> >> /path/prefix/b [/part-1, /part-2, etc]
> >> /path/prefix/f [/part-1, /part-2, etc]
> >>
> >> How would you do that?
> >>
> >> I suspect I need to use saveAsNewAPIHadoopFile or saveAsHadoopFile along
> >> with the MultipleTextOutputFormat output format class, but I’m not sure
> how.
> >>
> >> By the way, there is a very similar question to this here on Stack
> >> Overflow.
> >>
> >> Nick
> >>
> >>
> >> ________________________________
> >> View this message in context: Write 1 RDD to multiple output paths in
> one
> >> go
> >> Sent from the Apache Spark User List mailing list archive at Nabble.com.
> >
> >
>

Reply via email to