Yeah I agree that might have been nicer, but I think for consistency
with the input API's maybe we should do the same thing. We can also
give an example of how to clone sc.hadoopConfiguration and then set
some new values:

val conf = sc.hadoopConfiguration.clone()
  .set("k1", "v1")
  .set("k2", "v2")

val rdd = sc.objectFile(..., conf)

I have no idea if that's the correct syntax, but something like that
seems almost as easy as passing a hashmap with deltas.

- Patrick

On Wed, Mar 25, 2015 at 6:34 AM, Koert Kuipers <ko...@tresata.com> wrote:
> my personal preference would be something like a Map[String, String] that
> only reflects the changes you want to make the Configuration for the given
> input/output format (so system wide defaults continue to come from
> sc.hadoopConfiguration), similarly to what cascading/scalding did, but am
> arbitrary Configuration will work too.
>
> i will make a jira and pullreq when i have some time.
>
>
>
> On Wed, Mar 25, 2015 at 1:23 AM, Patrick Wendell <pwend...@gmail.com> wrote:
>>
>> I see - if you look, in the saving functions we have the option for
>> the user to pass an arbitrary Configuration.
>>
>>
>> https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala#L894
>>
>> It seems fine to have the same option for the loading functions, if
>> it's easy to just pass this config into the input format.
>>
>>
>>
>> On Tue, Mar 24, 2015 at 3:46 PM, Koert Kuipers <ko...@tresata.com> wrote:
>> > the (compression) codec parameter that is now part of many saveAs...
>> > methods
>> > came from a similar need. see SPARK-763
>> > hadoop has many options like this. you either going to have to allow
>> > many
>> > more of these optional arguments to all the methods that read from
>> > hadoop
>> > inputformats and write to hadoop outputformats, or you force people to
>> > re-create these methods using HadoopRDD, i think (if thats even
>> > possible).
>> >
>> > On Tue, Mar 24, 2015 at 6:40 PM, Koert Kuipers <ko...@tresata.com>
>> > wrote:
>> >>
>> >> i would like to use objectFile with some tweaks to the hadoop conf.
>> >> currently there is no way to do that, except recreating objectFile
>> >> myself.
>> >> and some of the code objectFile uses i have no access to, since its
>> >> private
>> >> to spark.
>> >>
>> >>
>> >> On Tue, Mar 24, 2015 at 2:59 PM, Patrick Wendell <pwend...@gmail.com>
>> >> wrote:
>> >>>
>> >>> Yeah - to Nick's point, I think the way to do this is to pass in a
>> >>> custom conf when you create a Hadoop RDD (that's AFAIK why the conf
>> >>> field is there). Is there anything you can't do with that feature?
>> >>>
>> >>> On Tue, Mar 24, 2015 at 11:50 AM, Nick Pentreath
>> >>> <nick.pentre...@gmail.com> wrote:
>> >>> > Imran, on your point to read multiple files together in a partition,
>> >>> > is
>> >>> > it
>> >>> > not simpler to use the approach of copy Hadoop conf and set per-RDD
>> >>> > settings for min split to control the input size per partition,
>> >>> > together
>> >>> > with something like CombineFileInputFormat?
>> >>> >
>> >>> > On Tue, Mar 24, 2015 at 5:28 PM, Imran Rashid <iras...@cloudera.com>
>> >>> > wrote:
>> >>> >
>> >>> >> I think this would be a great addition, I totally agree that you
>> >>> >> need
>> >>> >> to be
>> >>> >> able to set these at a finer context than just the SparkContext.
>> >>> >>
>> >>> >> Just to play devil's advocate, though -- the alternative is for you
>> >>> >> just
>> >>> >> subclass HadoopRDD yourself, or make a totally new RDD, and then
>> >>> >> you
>> >>> >> could
>> >>> >> expose whatever you need.  Why is this solution better?  IMO the
>> >>> >> criteria
>> >>> >> are:
>> >>> >> (a) common operations
>> >>> >> (b) error-prone / difficult to implement
>> >>> >> (c) non-obvious, but important for performance
>> >>> >>
>> >>> >> I think this case fits (a) & (c), so I think its still worthwhile.
>> >>> >> But its
>> >>> >> also worth asking whether or not its too difficult for a user to
>> >>> >> extend
>> >>> >> HadoopRDD right now.  There have been several cases in the past
>> >>> >> week
>> >>> >> where
>> >>> >> we've suggested that a user should read from hdfs themselves (eg.,
>> >>> >> to
>> >>> >> read
>> >>> >> multiple files together in one partition) -- with*out* reusing the
>> >>> >> code in
>> >>> >> HadoopRDD, though they would lose things like the metric tracking &
>> >>> >> preferred locations you get from HadoopRDD.  Does HadoopRDD need to
>> >>> >> some
>> >>> >> refactoring to make that easier to do?  Or do we just need a good
>> >>> >> example?
>> >>> >>
>> >>> >> Imran
>> >>> >>
>> >>> >> (sorry for hijacking your thread, Koert)
>> >>> >>
>> >>> >>
>> >>> >>
>> >>> >> On Mon, Mar 23, 2015 at 3:52 PM, Koert Kuipers <ko...@tresata.com>
>> >>> >> wrote:
>> >>> >>
>> >>> >> > see email below. reynold suggested i send it to dev instead of
>> >>> >> > user
>> >>> >> >
>> >>> >> > ---------- Forwarded message ----------
>> >>> >> > From: Koert Kuipers <ko...@tresata.com>
>> >>> >> > Date: Mon, Mar 23, 2015 at 4:36 PM
>> >>> >> > Subject: hadoop input/output format advanced control
>> >>> >> > To: "u...@spark.apache.org" <u...@spark.apache.org>
>> >>> >> >
>> >>> >> >
>> >>> >> > currently its pretty hard to control the Hadoop Input/Output
>> >>> >> > formats
>> >>> >> > used
>> >>> >> > in Spark. The conventions seems to be to add extra parameters to
>> >>> >> > all
>> >>> >> > methods and then somewhere deep inside the code (for example in
>> >>> >> > PairRDDFunctions.saveAsHadoopFile) all these parameters get
>> >>> >> > translated
>> >>> >> into
>> >>> >> > settings on the Hadoop Configuration object.
>> >>> >> >
>> >>> >> > for example for compression i see "codec: Option[Class[_ <:
>> >>> >> > CompressionCodec]] = None" added to a bunch of methods.
>> >>> >> >
>> >>> >> > how scalable is this solution really?
>> >>> >> >
>> >>> >> > for example i need to read from a hadoop dataset and i dont want
>> >>> >> > the
>> >>> >> input
>> >>> >> > (part) files to get split up. the way to do this is to set
>> >>> >> > "mapred.min.split.size". now i dont want to set this at the level
>> >>> >> > of
>> >>> >> > the
>> >>> >> > SparkContext (which can be done), since i dont want it to apply
>> >>> >> > to
>> >>> >> > input
>> >>> >> > formats in general. i want it to apply to just this one specific
>> >>> >> > input
>> >>> >> > dataset i need to read. which leaves me with no options
>> >>> >> > currently. i
>> >>> >> could
>> >>> >> > go add yet another input parameter to all the methods
>> >>> >> > (SparkContext.textFile, SparkContext.hadoopFile,
>> >>> >> > SparkContext.objectFile,
>> >>> >> > etc.). but that seems ineffective.
>> >>> >> >
>> >>> >> > why can we not expose a Map[String, String] or some other generic
>> >>> >> > way to
>> >>> >> > manipulate settings for hadoop input/output formats? it would
>> >>> >> > require
>> >>> >> > adding one more parameter to all methods to deal with hadoop
>> >>> >> > input/output
>> >>> >> > formats, but after that its done. one parameter to rule them
>> >>> >> > all....
>> >>> >> >
>> >>> >> > then i could do:
>> >>> >> > val x = sc.textFile("/some/path", formatSettings =
>> >>> >> > Map("mapred.min.split.size" -> "12345"))
>> >>> >> >
>> >>> >> > or
>> >>> >> > rdd.saveAsTextFile("/some/path, formatSettings =
>> >>> >> > Map(mapred.output.compress" -> "true",
>> >>> >> > "mapred.output.compression.codec"
>> >>> >> ->
>> >>> >> > "somecodec"))
>> >>> >> >
>> >>> >>
>> >>>
>> >>> ---------------------------------------------------------------------
>> >>> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
>> >>> For additional commands, e-mail: dev-h...@spark.apache.org
>> >>>
>> >>
>> >
>
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org

Reply via email to