You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Jerry Lam <ch...@gmail.com> on 2015/10/27 18:43:39 UTC

[Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Hi Spark users and developers,

Anyone experiences issues in setting hadoop configurations after
SparkContext is initialized? I'm using Spark 1.5.1.

I'm trying to use s3a which requires access and secret key set into hadoop
configuration. I tried to set the properties in the hadoop configuration
from sparktcontext.

sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)

val sqlContext = new SQLContext(sc)
val df = sqlContext.read.parquet("s3a://parquetfiles")

So far so good, I saw a job has been submitted to get the parquet schema
and it returns successfully.

and then I tried to do:

df.count

This failed with AmazonClientException:

com.amazonaws.AmazonClientException: Unable to load AWS credentials from
any provider in the chain
at
com.amazonaws.auth.AWSCredentialsProviderChain.getCredentials(AWSCredentialsProviderChain.java:117)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3521)
at
com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1031)
at
com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:994)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:297)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2596)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:91)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2630)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2612)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
at
org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:384)
at
org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:157)
at
org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
at
org.apache.spark.rdd.SqlNewHadoopRDD$$anon$1.<init>(SqlNewHadoopRDD.scala:155)
at org.apache.spark.rdd.SqlNewHadoopRDD.compute(SqlNewHadoopRDD.scala:120)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
at org.apache.spark.scheduler.Task.run(Task.scala:88)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)

Any idea why it can read the schema from the parquet file but not
processing the file? It feels like the hadoop configuration is not sent to
the executor for some reasons...

Thanks,

Jerry

RE: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Posted by "Cheng, Hao" <ha...@intel.com>.
Hi Jerry, I’ve filed a bug in jira, and also the fixing

https://issues.apache.org/jira/browse/SPARK-11364

It will be great appreciated if you can verify the PR with your case.

Thanks,
Hao

From: Cheng, Hao [mailto:hao.cheng@intel.com]
Sent: Wednesday, October 28, 2015 8:51 AM
To: Jerry Lam; Marcelo Vanzin
Cc: user@spark.apache.org
Subject: RE: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

After a draft glance, seems a bug in Spark SQL, do you mind to create a jira for this? And then I can start to fix it.

Thanks,
Hao

From: Jerry Lam [mailto:chilinglam@gmail.com]
Sent: Wednesday, October 28, 2015 3:13 AM
To: Marcelo Vanzin
Cc: user@spark.apache.org<ma...@spark.apache.org>
Subject: Re: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Hi Marcelo,

I tried setting the properties before instantiating spark context via SparkConf. It works fine.
Originally, the code I have read hadoop configurations from hdfs-site.xml which works perfectly fine as well.
Therefore, can I conclude that sparkContext.hadoopConfiguration.set("key", "value") does not propagate through all SQL jobs within the same SparkContext? I haven't try with Spark Core so I cannot tell.

Is there a workaround given it seems to be broken? I need to do this programmatically after the SparkContext is instantiated not before...

Best Regards,

Jerry

On Tue, Oct 27, 2015 at 2:30 PM, Marcelo Vanzin <va...@cloudera.com>> wrote:
If setting the values in SparkConf works, there's probably some bug in
the SQL code; e.g. creating a new Configuration object instead of
using the one in SparkContext. But I'm not really familiar with that
code.

On Tue, Oct 27, 2015 at 11:22 AM, Jerry Lam <ch...@gmail.com>> wrote:
> Hi Marcelo,
>
> Thanks for the advice. I understand that we could set the configurations
> before creating SparkContext. My question is
> SparkContext.hadoopConfiguration.set("key","value") doesn't seem to
> propagate to all subsequent SQLContext jobs. Note that I mentioned I can
> load the parquet file but I cannot perform a count on the parquet file
> because of the AmazonClientException. It means that the credential is used
> during the loading of the parquet but not when we are processing the parquet
> file. How this can happen?
>
> Best Regards,
>
> Jerry
>
>
> On Tue, Oct 27, 2015 at 2:05 PM, Marcelo Vanzin <va...@cloudera.com>> wrote:
>>
>> On Tue, Oct 27, 2015 at 10:43 AM, Jerry Lam <ch...@gmail.com>> wrote:
>> > Anyone experiences issues in setting hadoop configurations after
>> > SparkContext is initialized? I'm using Spark 1.5.1.
>> >
>> > I'm trying to use s3a which requires access and secret key set into
>> > hadoop
>> > configuration. I tried to set the properties in the hadoop configuration
>> > from sparktcontext.
>> >
>> > sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
>> > sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)
>>
>> Try setting "spark.hadoop.fs.s3a.access.key" and
>> "spark.hadoop.fs.s3a.secret.key" in your SparkConf before creating the
>> SparkContext.
>>
>> --
>> Marcelo
>
>

--
Marcelo


RE: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Posted by "Cheng, Hao" <ha...@intel.com>.
After a draft glance, seems a bug in Spark SQL, do you mind to create a jira for this? And then I can start to fix it.

Thanks,
Hao

From: Jerry Lam [mailto:chilinglam@gmail.com]
Sent: Wednesday, October 28, 2015 3:13 AM
To: Marcelo Vanzin
Cc: user@spark.apache.org
Subject: Re: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Hi Marcelo,

I tried setting the properties before instantiating spark context via SparkConf. It works fine.
Originally, the code I have read hadoop configurations from hdfs-site.xml which works perfectly fine as well.
Therefore, can I conclude that sparkContext.hadoopConfiguration.set("key", "value") does not propagate through all SQL jobs within the same SparkContext? I haven't try with Spark Core so I cannot tell.

Is there a workaround given it seems to be broken? I need to do this programmatically after the SparkContext is instantiated not before...

Best Regards,

Jerry

On Tue, Oct 27, 2015 at 2:30 PM, Marcelo Vanzin <va...@cloudera.com>> wrote:
If setting the values in SparkConf works, there's probably some bug in
the SQL code; e.g. creating a new Configuration object instead of
using the one in SparkContext. But I'm not really familiar with that
code.

On Tue, Oct 27, 2015 at 11:22 AM, Jerry Lam <ch...@gmail.com>> wrote:
> Hi Marcelo,
>
> Thanks for the advice. I understand that we could set the configurations
> before creating SparkContext. My question is
> SparkContext.hadoopConfiguration.set("key","value") doesn't seem to
> propagate to all subsequent SQLContext jobs. Note that I mentioned I can
> load the parquet file but I cannot perform a count on the parquet file
> because of the AmazonClientException. It means that the credential is used
> during the loading of the parquet but not when we are processing the parquet
> file. How this can happen?
>
> Best Regards,
>
> Jerry
>
>
> On Tue, Oct 27, 2015 at 2:05 PM, Marcelo Vanzin <va...@cloudera.com>> wrote:
>>
>> On Tue, Oct 27, 2015 at 10:43 AM, Jerry Lam <ch...@gmail.com>> wrote:
>> > Anyone experiences issues in setting hadoop configurations after
>> > SparkContext is initialized? I'm using Spark 1.5.1.
>> >
>> > I'm trying to use s3a which requires access and secret key set into
>> > hadoop
>> > configuration. I tried to set the properties in the hadoop configuration
>> > from sparktcontext.
>> >
>> > sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
>> > sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)
>>
>> Try setting "spark.hadoop.fs.s3a.access.key" and
>> "spark.hadoop.fs.s3a.secret.key" in your SparkConf before creating the
>> SparkContext.
>>
>> --
>> Marcelo
>
>


--
Marcelo


Re: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Posted by Jerry Lam <ch...@gmail.com>.
Hi Marcelo,

I tried setting the properties before instantiating spark context via
SparkConf. It works fine.
Originally, the code I have read hadoop configurations from hdfs-site.xml
which works perfectly fine as well.
Therefore, can I conclude that sparkContext.hadoopConfiguration.set("key",
"value") does not propagate through all SQL jobs within the same
SparkContext? I haven't try with Spark Core so I cannot tell.

Is there a workaround given it seems to be broken? I need to do this
programmatically after the SparkContext is instantiated not before...

Best Regards,

Jerry

On Tue, Oct 27, 2015 at 2:30 PM, Marcelo Vanzin <va...@cloudera.com> wrote:

> If setting the values in SparkConf works, there's probably some bug in
> the SQL code; e.g. creating a new Configuration object instead of
> using the one in SparkContext. But I'm not really familiar with that
> code.
>
> On Tue, Oct 27, 2015 at 11:22 AM, Jerry Lam <ch...@gmail.com> wrote:
> > Hi Marcelo,
> >
> > Thanks for the advice. I understand that we could set the configurations
> > before creating SparkContext. My question is
> > SparkContext.hadoopConfiguration.set("key","value") doesn't seem to
> > propagate to all subsequent SQLContext jobs. Note that I mentioned I can
> > load the parquet file but I cannot perform a count on the parquet file
> > because of the AmazonClientException. It means that the credential is
> used
> > during the loading of the parquet but not when we are processing the
> parquet
> > file. How this can happen?
> >
> > Best Regards,
> >
> > Jerry
> >
> >
> > On Tue, Oct 27, 2015 at 2:05 PM, Marcelo Vanzin <va...@cloudera.com>
> wrote:
> >>
> >> On Tue, Oct 27, 2015 at 10:43 AM, Jerry Lam <ch...@gmail.com>
> wrote:
> >> > Anyone experiences issues in setting hadoop configurations after
> >> > SparkContext is initialized? I'm using Spark 1.5.1.
> >> >
> >> > I'm trying to use s3a which requires access and secret key set into
> >> > hadoop
> >> > configuration. I tried to set the properties in the hadoop
> configuration
> >> > from sparktcontext.
> >> >
> >> > sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
> >> > sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)
> >>
> >> Try setting "spark.hadoop.fs.s3a.access.key" and
> >> "spark.hadoop.fs.s3a.secret.key" in your SparkConf before creating the
> >> SparkContext.
> >>
> >> --
> >> Marcelo
> >
> >
>
>
>
> --
> Marcelo
>

Re: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Posted by Marcelo Vanzin <va...@cloudera.com>.
If setting the values in SparkConf works, there's probably some bug in
the SQL code; e.g. creating a new Configuration object instead of
using the one in SparkContext. But I'm not really familiar with that
code.

On Tue, Oct 27, 2015 at 11:22 AM, Jerry Lam <ch...@gmail.com> wrote:
> Hi Marcelo,
>
> Thanks for the advice. I understand that we could set the configurations
> before creating SparkContext. My question is
> SparkContext.hadoopConfiguration.set("key","value") doesn't seem to
> propagate to all subsequent SQLContext jobs. Note that I mentioned I can
> load the parquet file but I cannot perform a count on the parquet file
> because of the AmazonClientException. It means that the credential is used
> during the loading of the parquet but not when we are processing the parquet
> file. How this can happen?
>
> Best Regards,
>
> Jerry
>
>
> On Tue, Oct 27, 2015 at 2:05 PM, Marcelo Vanzin <va...@cloudera.com> wrote:
>>
>> On Tue, Oct 27, 2015 at 10:43 AM, Jerry Lam <ch...@gmail.com> wrote:
>> > Anyone experiences issues in setting hadoop configurations after
>> > SparkContext is initialized? I'm using Spark 1.5.1.
>> >
>> > I'm trying to use s3a which requires access and secret key set into
>> > hadoop
>> > configuration. I tried to set the properties in the hadoop configuration
>> > from sparktcontext.
>> >
>> > sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
>> > sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)
>>
>> Try setting "spark.hadoop.fs.s3a.access.key" and
>> "spark.hadoop.fs.s3a.secret.key" in your SparkConf before creating the
>> SparkContext.
>>
>> --
>> Marcelo
>
>



-- 
Marcelo

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org


Re: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Posted by Jerry Lam <ch...@gmail.com>.
Hi Marcelo,

Thanks for the advice. I understand that we could set the configurations
before creating SparkContext. My question is
SparkContext.hadoopConfiguration.set("key","value") doesn't seem to
propagate to all subsequent SQLContext jobs. Note that I mentioned I can
load the parquet file but I cannot perform a count on the parquet file
because of the AmazonClientException. It means that the credential is used
during the loading of the parquet but not when we are processing the
parquet file. How this can happen?

Best Regards,

Jerry


On Tue, Oct 27, 2015 at 2:05 PM, Marcelo Vanzin <va...@cloudera.com> wrote:

> On Tue, Oct 27, 2015 at 10:43 AM, Jerry Lam <ch...@gmail.com> wrote:
> > Anyone experiences issues in setting hadoop configurations after
> > SparkContext is initialized? I'm using Spark 1.5.1.
> >
> > I'm trying to use s3a which requires access and secret key set into
> hadoop
> > configuration. I tried to set the properties in the hadoop configuration
> > from sparktcontext.
> >
> > sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
> > sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)
>
> Try setting "spark.hadoop.fs.s3a.access.key" and
> "spark.hadoop.fs.s3a.secret.key" in your SparkConf before creating the
> SparkContext.
>
> --
> Marcelo
>

Re: [Spark-SQL]: Unable to propagate hadoop configuration after SparkContext is initialized

Posted by Marcelo Vanzin <va...@cloudera.com>.
On Tue, Oct 27, 2015 at 10:43 AM, Jerry Lam <ch...@gmail.com> wrote:
> Anyone experiences issues in setting hadoop configurations after
> SparkContext is initialized? I'm using Spark 1.5.1.
>
> I'm trying to use s3a which requires access and secret key set into hadoop
> configuration. I tried to set the properties in the hadoop configuration
> from sparktcontext.
>
> sc.hadoopConfiguration.set("fs.s3a.access.key", AWSAccessKeyId)
> sc.hadoopConfiguration.set("fs.s3a.secret.key", AWSSecretKey)

Try setting "spark.hadoop.fs.s3a.access.key" and
"spark.hadoop.fs.s3a.secret.key" in your SparkConf before creating the
SparkContext.

-- 
Marcelo

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org