Reynold, thanks for this! At Palantir we're heavy users of the Java APIs
and appreciate being able to stop hacking around with fake ClassTags :)

Regarding this specific proposal, is the contract of RecordReader#get
intended to be that it returns a fresh object each time? Or is it allowed
to mutate a fixed object and return a pointer to it each time?

Put another way, is a caller supposed to clone the output of get() if they
want to use it later?

Punya
On Tue, Apr 21, 2015 at 4:35 PM Reynold Xin <r...@databricks.com> wrote:

> I created a pull request last night for a new InputSource API that is
> essentially a stripped down version of the RDD API for providing data into
> Spark. Would be great to hear the community's feedback.
>
> Spark currently has two de facto input source API:
> 1. RDD
> 2. Hadoop MapReduce InputFormat
>
> Neither of the above is ideal:
>
> 1. RDD: It is hard for Java developers to implement RDD, given the implicit
> class tags. In addition, the RDD API depends on Scala's runtime library,
> which does not preserve binary compatibility across Scala versions. If a
> developer chooses Java to implement an input source, it would be great if
> that input source can be binary compatible in years to come.
>
> 2. Hadoop InputFormat: The Hadoop InputFormat API is overly restrictive.
> For example, it forces key-value semantics, and does not support running
> arbitrary code on the driver side (an example of why this is useful is
> broadcast). In addition, it is somewhat awkward to tell developers that in
> order to implement an input source for Spark, they should learn the Hadoop
> MapReduce API first.
>
>
> My patch creates a new InputSource interface, described by:
>
> - an array of InputPartition that specifies the data partitioning
> - a RecordReader that specifies how data on each partition can be read
>
> This interface is similar to Hadoop's InputFormat, except that there is no
> explicit key/value separation.
>
>
> JIRA ticket: https://issues.apache.org/jira/browse/SPARK-7025
> Pull request: https://github.com/apache/spark/pull/5603
>

Reply via email to