Re: Hadoop configuration for checkpointing

2014-11-04 Thread Cody Koeninger
Opened https://issues.apache.org/jira/browse/SPARK-4229 Sent a PR https://github.com/apache/spark/pull/3102 On Tue, Nov 4, 2014 at 11:48 AM, Marcelo Vanzin wrote: > On Tue, Nov 4, 2014 at 9:34 AM, Cody Koeninger wrote: > > 2. Is there a reason StreamingContext.getOrCreate defaults to a blank

Re: Hadoop configuration for checkpointing

2014-11-04 Thread Sean Owen
Let me crash this thread to suggest this *might* be related to this problem I'm trying to solve: https://issues.apache.org/jira/browse/SPARK-4196 Basically the question there is: this blank Configuration object gets made on the driver in the saveAsNewAPIHadoopFiles call, and seems to need to be se

Re: Hadoop configuration for checkpointing

2014-11-04 Thread Marcelo Vanzin
On Tue, Nov 4, 2014 at 9:34 AM, Cody Koeninger wrote: > 2. Is there a reason StreamingContext.getOrCreate defaults to a blank > hadoop configuration rather than > org.apache.spark.deploy.SparkHadoopUtil.get.conf, > which would pull values from spark config? This is probably something I overlooke

Hadoop configuration for checkpointing

2014-11-04 Thread Cody Koeninger
3 quick questions, then some background: 1. Is there a reason not to document the fact that spark.hadoop.* is copied from spark config into hadoop config? 2. Is there a reason StreamingContext.getOrCreate defaults to a blank hadoop configuration rather than org.apache.spark.deploy.SparkHadoopUt